Merge branch 'master' into arcadia

This commit is contained in:
Ivan Lezhankin 2020-02-17 17:37:10 +03:00
commit 102dad0ca0
150 changed files with 3174 additions and 649 deletions

4
.gitmodules vendored
View File

@ -104,6 +104,10 @@
[submodule "contrib/sparsehash-c11"]
path = contrib/sparsehash-c11
url = https://github.com/sparsehash/sparsehash-c11.git
[submodule "contrib/grpc"]
path = contrib/grpc
url = https://github.com/grpc/grpc.git
branch = v1.25.0
[submodule "contrib/aws"]
path = contrib/aws
url = https://github.com/aws/aws-sdk-cpp.git

View File

@ -344,6 +344,7 @@ endif()
include (cmake/find/libxml2.cmake)
include (cmake/find/brotli.cmake)
include (cmake/find/protobuf.cmake)
include (cmake/find/grpc.cmake)
include (cmake/find/pdqsort.cmake)
include (cmake/find/hdfs3.cmake) # uses protobuf
include (cmake/find/s3.cmake)

View File

@ -138,7 +138,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
if (config.getBool("logger.console", false)
|| (!config.hasProperty("logger.console") && !is_daemon && is_tty))
{
bool color_enabled = config.getBool("logger.colored_console", false) && is_tty;
bool color_enabled = config.getBool("logger.color_terminal", true) && is_tty;
Poco::AutoPtr<OwnPatternFormatter> pf = new OwnPatternFormatter(this, OwnPatternFormatter::ADD_NOTHING, color_enabled);
Poco::AutoPtr<DB::OwnFormattingChannel> log = new DB::OwnFormattingChannel(pf, new Poco::ConsoleChannel);

View File

@ -2,40 +2,34 @@
#include <functional>
#include <optional>
#include <sys/time.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Common/HashTable/Hash.h>
#include <Interpreters/InternalTextLogsQueue.h>
#include <sys/time.h>
#include <Common/CurrentThread.h>
#include <common/getThreadId.h>
#include "Loggers.h"
static const char * setColor(UInt64 num)
static std::string setColor(UInt64 num)
{
/// ANSI escape sequences to set foreground font color in terminal.
/// Make a random RGB color that has constant brightness.
/// https://en.wikipedia.org/wiki/YCbCr
static constexpr auto num_colors = 12;
static const char * colors[num_colors] =
{
/// Black on black is meaningless
"\033[0;31m",
"\033[0;32m",
"\033[0;33m",
/// Low intense blue on black is too dark.
"\033[0;35m",
"\033[0;36m",
"\033[1;31m",
"\033[1;32m",
"\033[1;33m",
"\033[1;34m",
"\033[1;35m",
"\033[1;36m",
"\033[1m", /// Not as white but just as high intense - for readability on white background.
};
/// Note that this is darker than the middle relative luminance, see "Gamma_correction" and "Luma_(video)".
/// It still looks awesome.
UInt8 y = 128;
return colors[num % num_colors];
UInt8 cb = num % 256;
UInt8 cr = num / 256 % 256;
UInt8 r = std::max(0.0, std::min(255.0, y + 1.402 * (cr - 128)));
UInt8 g = std::max(0.0, std::min(255.0, y - 0.344136 * (cb - 128) - 0.714136 * (cr - 128)));
UInt8 b = std::max(0.0, std::min(255.0, y + 1.772 * (cb - 128)));
/// ANSI escape sequence to set 24-bit foreground font color in terminal.
return "\033[38;2;" + DB::toString(r) + ";" + DB::toString(g) + ";" + DB::toString(b) + "m";
}
static const char * setColorForLogPriority(int priority)
@ -104,7 +98,7 @@ void OwnPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ext,
writeCString(" [ ", wb);
if (color)
writeCString(setColor(intHash64(msg_ext.thread_id)), wb);
writeString(setColor(intHash64(msg_ext.thread_id)), wb);
DB::writeIntText(msg_ext.thread_id, wb);
if (color)
writeCString(resetColor(), wb);
@ -114,7 +108,7 @@ void OwnPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ext,
/// just to be convenient for various log parsers.
writeCString("{", wb);
if (color)
writeCString(setColor(std::hash<std::string>()(msg_ext.query_id)), wb);
writeString(setColor(std::hash<std::string>()(msg_ext.query_id)), wb);
DB::writeString(msg_ext.query_id, wb);
if (color)
writeCString(resetColor(), wb);

57
cmake/find/grpc.cmake Normal file
View File

@ -0,0 +1,57 @@
set(_PROTOBUF_PROTOC $<TARGET_FILE:protoc>)
set(_GRPC_CPP_PLUGIN_EXECUTABLE $<TARGET_FILE:grpc_cpp_plugin>)
function(PROTOBUF_GENERATE_GRPC_CPP SRCS HDRS)
if(NOT ARGN)
message(SEND_ERROR "Error: PROTOBUF_GENERATE_GRPC_CPP() called without any proto files")
return()
endif()
if(PROTOBUF_GENERATE_CPP_APPEND_PATH)
foreach(FIL ${ARGN})
get_filename_component(ABS_FIL ${FIL} ABSOLUTE)
get_filename_component(ABS_PATH ${ABS_FIL} PATH)
list(FIND _protobuf_include_path ${ABS_PATH} _contains_already)
if(${_contains_already} EQUAL -1)
list(APPEND _protobuf_include_path -I ${ABS_PATH})
endif()
endforeach()
else()
set(_protobuf_include_path -I ${CMAKE_CURRENT_SOURCE_DIR})
endif()
if(DEFINED PROTOBUF_IMPORT_DIRS)
foreach(DIR ${Protobuf_IMPORT_DIRS})
get_filename_component(ABS_PATH ${DIR} ABSOLUTE)
list(FIND _protobuf_include_path ${ABS_PATH} _contains_already)
if(${_contains_already} EQUAL -1)
list(APPEND _protobuf_include_path -I ${ABS_PATH})
endif()
endforeach()
endif()
set(${SRCS})
set(${HDRS})
foreach(FIL ${ARGN})
get_filename_component(ABS_FIL ${FIL} ABSOLUTE)
get_filename_component(FIL_WE ${FIL} NAME_WE)
list(APPEND ${SRCS} "${CMAKE_CURRENT_BINARY_DIR}/${FIL_WE}.grpc.pb.cc")
list(APPEND ${HDRS} "${CMAKE_CURRENT_BINARY_DIR}/${FIL_WE}.grpc.pb.h")
add_custom_command(
OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/${FIL_WE}.grpc.pb.cc"
"${CMAKE_CURRENT_BINARY_DIR}/${FIL_WE}.grpc.pb.h"
COMMAND ${_PROTOBUF_PROTOC}
ARGS --grpc_out=${CMAKE_CURRENT_BINARY_DIR}
--plugin=protoc-gen-grpc=${_GRPC_CPP_PLUGIN_EXECUTABLE}
${_protobuf_include_path} ${ABS_FIL}
DEPENDS ${ABS_FIL}
COMMENT "Running gRPC C++ protocol buffer compiler on ${FIL}"
VERBATIM)
endforeach()
set_source_files_properties(${${SRCS}} ${${HDRS}} PROPERTIES GENERATED TRUE)
set(${SRCS} ${${SRCS}} PARENT_SCOPE)
set(${HDRS} ${${HDRS}} PARENT_SCOPE)
endfunction()

View File

@ -336,4 +336,6 @@ if (USE_FASTOPS)
add_subdirectory (fastops-cmake)
endif()
add_subdirectory(grpc-cmake)
add_subdirectory(replxx-cmake)

1
contrib/grpc vendored Submodule

@ -0,0 +1 @@
Subproject commit c1d176528fd8da9dd4066d16554bcd216d29033f

File diff suppressed because it is too large Load Diff

View File

@ -76,6 +76,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request
}
else
{
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_SERVICE_UNAVAILABLE);
response.send() << message.rdbuf();
}
}

View File

@ -578,6 +578,25 @@ int Server::main(const std::vector<std::string> & /*args*/)
global_context->initializeTraceCollector();
#endif
/// Describe multiple reasons when query profiler cannot work.
#if !USE_UNWIND
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they cannot work without bundled unwind (stack unwinding) library.");
#endif
#if WITH_COVERAGE
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage.");
#endif
#if defined(SANITIZER)
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they cannot work under sanitizers"
" when two different stack unwinding methods will interfere with each other.");
#endif
if (!hasPHDRCache())
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created"
" (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe).");
global_context->setCurrentDatabase(default_database);
if (has_zookeeper && config().has("distributed_ddl"))

View File

@ -22,7 +22,6 @@ namespace
{
using IPAddress = Poco::Net::IPAddress;
using IPSubnet = AllowedClientHosts::IPSubnet;
const IPSubnet ALL_ADDRESSES{IPAddress{IPAddress::IPv6}, IPAddress{IPAddress::IPv6}};
/// Converts an address to IPv6.
/// The loopback address "127.0.0.1" (or any "127.x.y.z") is converted to "::1".

View File

@ -440,6 +440,8 @@ XMLDocumentPtr ConfigProcessor::processConfig(
zkutil::ZooKeeperNodeCache * zk_node_cache,
const zkutil::EventPtr & zk_changed_event)
{
LOG_DEBUG(log, "Processing configuration file '" + path + "'.");
XMLDocumentPtr config = dom_parser.parse(path);
std::vector<std::string> contributing_files;
@ -449,6 +451,8 @@ XMLDocumentPtr ConfigProcessor::processConfig(
{
try
{
LOG_DEBUG(log, "Merging configuration file '" + merge_file + "'.");
XMLDocumentPtr with = dom_parser.parse(merge_file);
merge(config, with);
contributing_files.push_back(merge_file);
@ -484,6 +488,8 @@ XMLDocumentPtr ConfigProcessor::processConfig(
}
if (!include_from_path.empty())
{
LOG_DEBUG(log, "Including configuration file '" + include_from_path + "'.");
contributing_files.push_back(include_from_path);
include_from = dom_parser.parse(include_from_path);
}
@ -613,6 +619,7 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
Poco::File(preprocessed_path_parent).createDirectories();
}
DOMWriter().writeNode(preprocessed_path, loaded_config.preprocessed_xml);
LOG_DEBUG(log, "Saved preprocessed configuration to '" << preprocessed_path << "'.");
}
catch (Poco::Exception & e)
{

View File

@ -95,6 +95,7 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_
* with_stacktrace - prints stack trace for DB::Exception.
* check_embedded_stacktrace - if DB::Exception has embedded stacktrace then
* only this stack trace will be printed.
* with_extra_info - add information about the filesystem in case of "No space left on device" and similar.
*/
std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace = false,
bool with_extra_info = true);

View File

@ -384,7 +384,6 @@ struct Settings : public SettingsCollection<Settings>
\
M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \
M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \
M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \
M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \
\
@ -408,6 +407,8 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, merge_tree_uniform_read_distribution, true, "Obsolete setting, does nothing. Will be removed after 2020-05-20", 0) \
M(SettingUInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(SettingUInt64, max_parser_depth, 1000, "Maximum parser depth.", 0) \
M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS)

View File

@ -12,10 +12,12 @@
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeInterval.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/Native.h>
#include <DataTypes/NumberTraits.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnAggregateFunction.h>
#include "IFunctionImpl.h"
@ -59,6 +61,7 @@ template <typename A, typename B, typename Op, typename ResultType_ = typename O
struct BinaryOperationImplBase
{
using ResultType = ResultType_;
static const constexpr bool allow_fixed_string = false;
static void NO_INLINE vector_vector(const PaddedPODArray<A> & a, const PaddedPODArray<B> & b, PaddedPODArray<ResultType> & c)
{
@ -87,6 +90,33 @@ struct BinaryOperationImplBase
}
};
template <typename Op>
struct FixedStringOperationImpl
{
static void NO_INLINE vector_vector(const ColumnFixedString::Chars & a, const ColumnFixedString::Chars & b, ColumnFixedString::Chars & c)
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = Op::template apply<UInt8>(a[i], b[i]);
}
static void NO_INLINE vector_constant(const ColumnFixedString::Chars & a, const ColumnFixedString::Chars & b, ColumnFixedString::Chars & c)
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = Op::template apply<UInt8>(a[i], b[i % b.size()]);
}
static void NO_INLINE constant_vector(const ColumnFixedString::Chars & a, const ColumnFixedString::Chars & b, ColumnFixedString::Chars & c)
{
size_t size = b.size();
for (size_t i = 0; i < size; ++i)
c[i] = Op::template apply<UInt8>(a[i % a.size()], b[i]);
}
};
template <typename A, typename B, typename Op, typename ResultType = typename Op::ResultType>
struct BinaryOperationImpl : BinaryOperationImplBase<A, B, Op, ResultType>
{
@ -463,7 +493,8 @@ class FunctionBinaryArithmetic : public IFunction
DataTypeDateTime,
DataTypeDecimal<Decimal32>,
DataTypeDecimal<Decimal64>,
DataTypeDecimal<Decimal128>
DataTypeDecimal<Decimal128>,
DataTypeFixedString
>(type, std::forward<F>(f));
}
@ -719,27 +750,43 @@ public:
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
if constexpr (!std::is_same_v<ResultDataType, InvalidType>)
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> || std::is_same_v<DataTypeFixedString, RightDataType>)
{
if constexpr (IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType>)
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
return false;
else if constexpr (std::is_same_v<LeftDataType, RightDataType>)
{
constexpr bool is_multiply = std::is_same_v<Op<UInt8, UInt8>, MultiplyImpl<UInt8, UInt8>>;
constexpr bool is_division = std::is_same_v<Op<UInt8, UInt8>, DivideFloatingImpl<UInt8, UInt8>> ||
std::is_same_v<Op<UInt8, UInt8>, DivideIntegralImpl<UInt8, UInt8>> ||
std::is_same_v<Op<UInt8, UInt8>, DivideIntegralOrZeroImpl<UInt8, UInt8>>;
ResultDataType result_type = decimalResultType(left, right, is_multiply, is_division);
type_res = std::make_shared<ResultDataType>(result_type.getPrecision(), result_type.getScale());
if (left.getN() == right.getN())
{
type_res = std::make_shared<LeftDataType>(left.getN());
return true;
}
}
else if constexpr (IsDataTypeDecimal<LeftDataType>)
type_res = std::make_shared<LeftDataType>(left.getPrecision(), left.getScale());
else if constexpr (IsDataTypeDecimal<RightDataType>)
type_res = std::make_shared<RightDataType>(right.getPrecision(), right.getScale());
else
type_res = std::make_shared<ResultDataType>();
return true;
}
else
{
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
if constexpr (!std::is_same_v<ResultDataType, InvalidType>)
{
if constexpr (IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType>)
{
constexpr bool is_multiply = std::is_same_v<Op<UInt8, UInt8>, MultiplyImpl<UInt8, UInt8>>;
constexpr bool is_division = std::is_same_v<Op<UInt8, UInt8>, DivideFloatingImpl<UInt8, UInt8>> ||
std::is_same_v<Op<UInt8, UInt8>, DivideIntegralImpl<UInt8, UInt8>> ||
std::is_same_v<Op<UInt8, UInt8>, DivideIntegralOrZeroImpl<UInt8, UInt8>>;
ResultDataType result_type = decimalResultType(left, right, is_multiply, is_division);
type_res = std::make_shared<ResultDataType>(result_type.getPrecision(), result_type.getScale());
}
else if constexpr (IsDataTypeDecimal<LeftDataType>)
type_res = std::make_shared<LeftDataType>(left.getPrecision(), left.getScale());
else if constexpr (IsDataTypeDecimal<RightDataType>)
type_res = std::make_shared<RightDataType>(right.getPrecision(), right.getScale());
else
type_res = std::make_shared<ResultDataType>();
return true;
}
}
return false;
});
if (!valid)
@ -748,7 +795,206 @@ public:
return type_res;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
bool executeFixedString(Block & block, const ColumnNumbers & arguments, size_t result)
{
using OpImpl = FixedStringOperationImpl<Op<UInt8, UInt8>>;
auto col_left_raw = block.getByPosition(arguments[0]).column.get();
auto col_right_raw = block.getByPosition(arguments[1]).column.get();
if (auto col_left_const = checkAndGetColumnConst<ColumnFixedString>(col_left_raw))
{
if (auto col_right_const = checkAndGetColumnConst<ColumnFixedString>(col_right_raw))
{
auto col_left = checkAndGetColumn<ColumnFixedString>(col_left_const->getDataColumn());
auto col_right = checkAndGetColumn<ColumnFixedString>(col_right_const->getDataColumn());
if (col_left->getN() != col_right->getN())
return false;
auto col_res = ColumnFixedString::create(col_left->getN());
auto & out_chars = col_res->getChars();
out_chars.resize(col_left->getN());
OpImpl::vector_vector(col_left->getChars(),
col_right->getChars(),
out_chars);
block.getByPosition(result).column = ColumnConst::create(std::move(col_res), block.rows());
return true;
}
}
bool is_left_column_const = checkAndGetColumnConst<ColumnFixedString>(col_left_raw) != nullptr;
bool is_right_column_const = checkAndGetColumnConst<ColumnFixedString>(col_right_raw) != nullptr;
auto col_left = is_left_column_const
? checkAndGetColumn<ColumnFixedString>(checkAndGetColumnConst<ColumnFixedString>(col_left_raw)->getDataColumn())
: checkAndGetColumn<ColumnFixedString>(col_left_raw);
auto col_right = is_right_column_const
? checkAndGetColumn<ColumnFixedString>(checkAndGetColumnConst<ColumnFixedString>(col_right_raw)->getDataColumn())
: checkAndGetColumn<ColumnFixedString>(col_right_raw);
if (col_left && col_right)
{
if (col_left->getN() != col_right->getN())
return false;
auto col_res = ColumnFixedString::create(col_left->getN());
auto & out_chars = col_res->getChars();
out_chars.resize((is_right_column_const ? col_left->size() : col_right->size()) * col_left->getN());
if (!is_left_column_const && !is_right_column_const)
{
OpImpl::vector_vector(col_left->getChars(),
col_right->getChars(),
out_chars);
}
else if (is_left_column_const)
{
OpImpl::constant_vector(col_left->getChars(),
col_right->getChars(),
out_chars);
}
else
{
OpImpl::vector_constant(col_left->getChars(),
col_right->getChars(),
out_chars);
}
block.getByPosition(result).column = std::move(col_res);
return true;
}
return false;
}
template<typename A, typename B>
bool executeNumeric(Block & block, const ColumnNumbers & arguments, size_t result [[maybe_unused]], const A & left, const B & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
if constexpr (!std::is_same_v<ResultDataType, InvalidType>)
{
constexpr bool result_is_decimal = IsDataTypeDecimal<LeftDataType> || IsDataTypeDecimal<RightDataType>;
constexpr bool is_multiply = std::is_same_v<Op<UInt8, UInt8>, MultiplyImpl<UInt8, UInt8>>;
constexpr bool is_division = std::is_same_v<Op<UInt8, UInt8>, DivideFloatingImpl<UInt8, UInt8>> ||
std::is_same_v<Op<UInt8, UInt8>, DivideIntegralImpl<UInt8, UInt8>> ||
std::is_same_v<Op<UInt8, UInt8>, DivideIntegralOrZeroImpl<UInt8, UInt8>>;
using T0 = typename LeftDataType::FieldType;
using T1 = typename RightDataType::FieldType;
using ResultType = typename ResultDataType::FieldType;
using ColVecT0 = std::conditional_t<IsDecimalNumber<T0>, ColumnDecimal<T0>, ColumnVector<T0>>;
using ColVecT1 = std::conditional_t<IsDecimalNumber<T1>, ColumnDecimal<T1>, ColumnVector<T1>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<ResultType>, ColumnDecimal<ResultType>, ColumnVector<ResultType>>;
/// Decimal operations need scale. Operations are on result type.
using OpImpl = std::conditional_t<IsDataTypeDecimal<ResultDataType>,
DecimalBinaryOperation<T0, T1, Op, ResultType>,
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>>;
auto col_left_raw = block.getByPosition(arguments[0]).column.get();
auto col_right_raw = block.getByPosition(arguments[1]).column.get();
if (auto col_left = checkAndGetColumnConst<ColVecT0>(col_left_raw))
{
if (auto col_right = checkAndGetColumnConst<ColVecT1>(col_right_raw))
{
/// the only case with a non-vector result
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
auto res = OpImpl::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>(),
scale_a, scale_b, check_decimal_overflow);
block.getByPosition(result).column =
ResultDataType(type.getPrecision(), type.getScale()).createColumnConst(
col_left->size(), toField(res, type.getScale()));
}
else
{
auto res = OpImpl::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>());
block.getByPosition(result).column = ResultDataType().createColumnConst(col_left->size(), toField(res));
}
return true;
}
}
typename ColVecResult::MutablePtr col_res = nullptr;
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
col_res = ColVecResult::create(0, type.getScale());
}
else
col_res = ColVecResult::create();
auto & vec_res = col_res->getData();
vec_res.resize(block.rows());
if (auto col_left_const = checkAndGetColumnConst<ColVecT0>(col_left_raw))
{
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
{
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
OpImpl::constant_vector(col_left_const->template getValue<T0>(), col_right->getData(), vec_res,
scale_a, scale_b, check_decimal_overflow);
}
else
OpImpl::constant_vector(col_left_const->template getValue<T0>(), col_right->getData(), vec_res);
}
else
return false;
}
else if (auto col_left = checkAndGetColumn<ColVecT0>(col_left_raw))
{
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
{
OpImpl::vector_vector(col_left->getData(), col_right->getData(), vec_res, scale_a, scale_b,
check_decimal_overflow);
}
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw))
{
OpImpl::vector_constant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res,
scale_a, scale_b, check_decimal_overflow);
}
else
return false;
}
else
{
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
OpImpl::vector_vector(col_left->getData(), col_right->getData(), vec_res);
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw))
OpImpl::vector_constant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res);
else
return false;
}
}
else
return false;
block.getByPosition(result).column = std::move(col_res);
return true;
}
return false;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
/// Special case when multiply aggregate function state
if (isAggregateMultiply(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type))
@ -777,132 +1023,15 @@ public:
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
if constexpr (!std::is_same_v<ResultDataType, InvalidType>)
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> || std::is_same_v<DataTypeFixedString, RightDataType>)
{
constexpr bool result_is_decimal = IsDataTypeDecimal<LeftDataType> || IsDataTypeDecimal<RightDataType>;
constexpr bool is_multiply = std::is_same_v<Op<UInt8, UInt8>, MultiplyImpl<UInt8, UInt8>>;
constexpr bool is_division = std::is_same_v<Op<UInt8, UInt8>, DivideFloatingImpl<UInt8, UInt8>> ||
std::is_same_v<Op<UInt8, UInt8>, DivideIntegralImpl<UInt8, UInt8>> ||
std::is_same_v<Op<UInt8, UInt8>, DivideIntegralOrZeroImpl<UInt8, UInt8>>;
using T0 = typename LeftDataType::FieldType;
using T1 = typename RightDataType::FieldType;
using ResultType = typename ResultDataType::FieldType;
using ColVecT0 = std::conditional_t<IsDecimalNumber<T0>, ColumnDecimal<T0>, ColumnVector<T0>>;
using ColVecT1 = std::conditional_t<IsDecimalNumber<T1>, ColumnDecimal<T1>, ColumnVector<T1>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<ResultType>, ColumnDecimal<ResultType>, ColumnVector<ResultType>>;
/// Decimal operations need scale. Operations are on result type.
using OpImpl = std::conditional_t<IsDataTypeDecimal<ResultDataType>,
DecimalBinaryOperation<T0, T1, Op, ResultType>,
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>>;
auto col_left_raw = block.getByPosition(arguments[0]).column.get();
auto col_right_raw = block.getByPosition(arguments[1]).column.get();
if (auto col_left = checkAndGetColumnConst<ColVecT0>(col_left_raw))
{
if (auto col_right = checkAndGetColumnConst<ColVecT1>(col_right_raw))
{
/// the only case with a non-vector result
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
auto res = OpImpl::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>(),
scale_a, scale_b, check_decimal_overflow);
block.getByPosition(result).column =
ResultDataType(type.getPrecision(), type.getScale()).createColumnConst(
col_left->size(), toField(res, type.getScale()));
}
else
{
auto res = OpImpl::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>());
block.getByPosition(result).column = ResultDataType().createColumnConst(col_left->size(), toField(res));
}
return true;
}
}
typename ColVecResult::MutablePtr col_res = nullptr;
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
col_res = ColVecResult::create(0, type.getScale());
}
else
col_res = ColVecResult::create();
auto & vec_res = col_res->getData();
vec_res.resize(block.rows());
if (auto col_left_const = checkAndGetColumnConst<ColVecT0>(col_left_raw))
{
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
{
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
OpImpl::constant_vector(col_left_const->template getValue<T0>(), col_right->getData(), vec_res,
scale_a, scale_b, check_decimal_overflow);
}
else
OpImpl::constant_vector(col_left_const->template getValue<T0>(), col_right->getData(), vec_res);
}
else
return false;
}
else if (auto col_left = checkAndGetColumn<ColVecT0>(col_left_raw))
{
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
{
OpImpl::vector_vector(col_left->getData(), col_right->getData(), vec_res, scale_a, scale_b,
check_decimal_overflow);
}
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw))
{
OpImpl::vector_constant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res,
scale_a, scale_b, check_decimal_overflow);
}
else
return false;
}
else
{
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
OpImpl::vector_vector(col_left->getData(), col_right->getData(), vec_res);
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw))
OpImpl::vector_constant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res);
else
return false;
}
}
else
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
return false;
block.getByPosition(result).column = std::move(col_res);
return true;
else
return executeFixedString(block, arguments, result);
}
return false;
else
return executeNumeric(block, arguments, result, left, right);
});
if (!valid)
throw Exception(getName() + "'s arguments do not match the expected data types", ErrorCodes::LOGICAL_ERROR);
@ -915,9 +1044,14 @@ public:
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
return !std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType> && OpSpec::compilable;
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> || std::is_same_v<DataTypeFixedString, RightDataType>)
return false;
else
{
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
return !std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType> && OpSpec::compilable;
}
});
}
@ -928,16 +1062,19 @@ public:
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
if constexpr (!std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType> && OpSpec::compilable)
if constexpr (!std::is_same_v<DataTypeFixedString, LeftDataType> && !std::is_same_v<DataTypeFixedString, RightDataType>)
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto type = std::make_shared<ResultDataType>();
auto * lval = nativeCast(b, types[0], values[0](), type);
auto * rval = nativeCast(b, types[1], values[1](), type);
result = OpSpec::compile(b, lval, rval, std::is_signed_v<typename ResultDataType::FieldType>);
return true;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
if constexpr (!std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType> && OpSpec::compilable)
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto type = std::make_shared<ResultDataType>();
auto * lval = nativeCast(b, types[0], values[0](), type);
auto * rval = nativeCast(b, types[1], values[1](), type);
result = OpSpec::compile(b, lval, rval, std::is_signed_v<typename ResultDataType::FieldType>);
return true;
}
}
return false;
});

View File

@ -142,8 +142,9 @@ private:
{
if (auto pos_col_const = checkAndGetColumnConst<ColumnVector<ValueType>>(block.getByPosition(arguments[i]).column.get()))
{
const auto pos = pos_col_const->template getValue<ValueType>();
mask = mask | (1 << pos);
const auto pos = pos_col_const->getUInt(0);
if (pos < 8 * sizeof(ValueType))
mask = mask | (ValueType(1) << pos);
}
else
{
@ -175,21 +176,22 @@ private:
}
template <typename PosType, typename ValueType>
bool addToMaskImpl(PaddedPODArray<ValueType> & mask, const IColumn * const pos_col_untyped)
bool NO_SANITIZE_UNDEFINED addToMaskImpl(PaddedPODArray<ValueType> & mask, const IColumn * const pos_col_untyped)
{
if (const auto pos_col = checkAndGetColumn<ColumnVector<PosType>>(pos_col_untyped))
{
const auto & pos = pos_col->getData();
for (const auto i : ext::range(0, mask.size()))
mask[i] = mask[i] | (1 << pos[i]);
if (pos[i] < 8 * sizeof(ValueType))
mask[i] = mask[i] | (ValueType(1) << pos[i]);
return true;
}
else if (const auto pos_col_const = checkAndGetColumnConst<ColumnVector<PosType>>(pos_col_untyped))
{
const auto & pos = pos_col_const->template getValue<PosType>();
const auto new_mask = 1 << pos;
const auto new_mask = pos < 8 * sizeof(ValueType) ? ValueType(1) << pos : 0;
for (const auto i : ext::range(0, mask.size()))
mask[i] = mask[i] | new_mask;

View File

@ -17,6 +17,10 @@ public:
static constexpr auto name = "joinGet";
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return true; }
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
String getName() const override { return name; }
@ -72,6 +76,9 @@ public:
FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override;
DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override;
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return true; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }

View File

@ -2,9 +2,11 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/Native.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnFixedString.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/castTypeToEither.h>
@ -51,6 +53,18 @@ struct UnaryOperationImpl
};
template <typename Op>
struct FixedStringUnaryOperationImpl
{
static void NO_INLINE vector(const ColumnFixedString::Chars & a, ColumnFixedString::Chars & c)
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = Op::apply(a[i]);
}
};
template <typename FunctionName>
struct FunctionUnaryArithmeticMonotonicity;
@ -65,6 +79,7 @@ template <template <typename> class Op, typename Name, bool is_injective>
class FunctionUnaryArithmetic : public IFunction
{
static constexpr bool allow_decimal = std::is_same_v<Op<Int8>, NegateImpl<Int8>> || std::is_same_v<Op<Int8>, AbsImpl<Int8>>;
static constexpr bool allow_fixed_string = Op<UInt8>::allow_fixed_string;
template <typename F>
static bool castType(const IDataType * type, F && f)
@ -82,7 +97,8 @@ class FunctionUnaryArithmetic : public IFunction
DataTypeFloat64,
DataTypeDecimal<Decimal32>,
DataTypeDecimal<Decimal64>,
DataTypeDecimal<Decimal128>
DataTypeDecimal<Decimal128>,
DataTypeFixedString
>(type, std::forward<F>(f));
}
@ -106,16 +122,25 @@ public:
bool valid = castType(arguments[0].get(), [&](const auto & type)
{
using DataType = std::decay_t<decltype(type)>;
using T0 = typename DataType::FieldType;
if constexpr (IsDataTypeDecimal<DataType>)
if constexpr (std::is_same_v<DataTypeFixedString, DataType>)
{
if constexpr (!allow_decimal)
if constexpr (!Op<DataTypeFixedString>::allow_fixed_string)
return false;
result = std::make_shared<DataType>(type.getPrecision(), type.getScale());
result = std::make_shared<DataType>(type.getN());
}
else
result = std::make_shared<DataTypeNumber<typename Op<T0>::ResultType>>();
{
using T0 = typename DataType::FieldType;
if constexpr (IsDataTypeDecimal<DataType>)
{
if constexpr (!allow_decimal)
return false;
result = std::make_shared<DataType>(type.getPrecision(), type.getScale());
}
else
result = std::make_shared<DataTypeNumber<typename Op<T0>::ResultType>>();
}
return true;
});
if (!valid)
@ -129,10 +154,25 @@ public:
bool valid = castType(block.getByPosition(arguments[0]).type.get(), [&](const auto & type)
{
using DataType = std::decay_t<decltype(type)>;
using T0 = typename DataType::FieldType;
if constexpr (IsDataTypeDecimal<DataType>)
if constexpr (std::is_same_v<DataTypeFixedString, DataType>)
{
if constexpr (allow_fixed_string)
{
if (auto col = checkAndGetColumn<ColumnFixedString>(block.getByPosition(arguments[0]).column.get()))
{
auto col_res = ColumnFixedString::create(col->getN());
auto & vec_res = col_res->getChars();
vec_res.resize(col->size() * col->getN());
FixedStringUnaryOperationImpl<Op<UInt8>>::vector(col->getChars(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
}
}
}
else if constexpr (IsDataTypeDecimal<DataType>)
{
using T0 = typename DataType::FieldType;
if constexpr (allow_decimal)
{
if (auto col = checkAndGetColumn<ColumnDecimal<T0>>(block.getByPosition(arguments[0]).column.get()))
@ -148,6 +188,7 @@ public:
}
else
{
using T0 = typename DataType::FieldType;
if (auto col = checkAndGetColumn<ColumnVector<T0>>(block.getByPosition(arguments[0]).column.get()))
{
auto col_res = ColumnVector<typename Op<T0>::ResultType>::create();
@ -171,7 +212,10 @@ public:
return castType(arguments[0].get(), [&](const auto & type)
{
using DataType = std::decay_t<decltype(type)>;
return !IsDataTypeDecimal<DataType> && Op<typename DataType::FieldType>::compilable;
if constexpr (std::is_same_v<DataTypeFixedString, DataType>)
return false;
else
return !IsDataTypeDecimal<DataType> && Op<typename DataType::FieldType>::compilable;
});
}
@ -181,14 +225,19 @@ public:
castType(types[0].get(), [&](const auto & type)
{
using DataType = std::decay_t<decltype(type)>;
using T0 = typename DataType::FieldType;
using T1 = typename Op<T0>::ResultType;
if constexpr (!std::is_same_v<T1, InvalidType> && !IsDataTypeDecimal<DataType> && Op<T0>::compilable)
if constexpr (std::is_same_v<DataTypeFixedString, DataType>)
return false;
else
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * v = nativeCast(b, types[0], values[0](), std::make_shared<DataTypeNumber<T1>>());
result = Op<T0>::compile(b, v, is_signed_v<T1>);
return true;
using T0 = typename DataType::FieldType;
using T1 = typename Op<T0>::ResultType;
if constexpr (!std::is_same_v<T1, InvalidType> && !IsDataTypeDecimal<DataType> && Op<T0>::compilable)
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * v = nativeCast(b, types[0], values[0](), std::make_shared<DataTypeNumber<T1>>());
result = Op<T0>::compile(b, v, is_signed_v<T1>);
return true;
}
}
return false;
});

View File

@ -261,7 +261,7 @@ using FunctionReinterpretAsDate = FunctionReinterpretStringAs<DataTypeDate,
using FunctionReinterpretAsDateTime = FunctionReinterpretStringAs<DataTypeDateTime, NameReinterpretAsDateTime>;
using FunctionReinterpretAsString = FunctionReinterpretAsStringImpl<NameReinterpretAsString>;
using FunctionReinterpretAsFixedString = FunctionReinterpretAsStringImpl<NameReinterpretAsFixedString>;
using FunctionReinterpretAsFixedString = FunctionReinterpretAsFixedStringImpl<NameReinterpretAsFixedString>;
}

View File

@ -10,6 +10,7 @@ template <typename A>
struct AbsImpl
{
using ResultType = std::conditional_t<IsDecimalNumber<A>, A, typename NumberTraits::ResultOfAbs<A>::Type>;
static const constexpr bool allow_fixed_string = false;
static inline NO_SANITIZE_UNDEFINED ResultType apply(A a)
{

View File

@ -9,6 +9,7 @@ template <typename A, typename B>
struct BitAndImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static constexpr const bool allow_fixed_string = true;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
@ -29,7 +30,7 @@ struct BitAndImpl
};
struct NameBitAnd { static constexpr auto name = "bitAnd"; };
using FunctionBitAnd = FunctionBinaryArithmetic<BitAndImpl, NameBitAnd>;
using FunctionBitAnd = FunctionBinaryArithmetic<BitAndImpl, NameBitAnd, true>;
void registerFunctionBitAnd(FunctionFactory & factory)
{

View File

@ -18,6 +18,7 @@ namespace DB
struct BitBoolMaskAndImpl
{
using ResultType = UInt8;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A left, B right)

View File

@ -18,6 +18,7 @@ namespace DB
struct BitBoolMaskOrImpl
{
using ResultType = UInt8;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A left, B right)

View File

@ -10,6 +10,7 @@ template <typename A>
struct BitCountImpl
{
using ResultType = UInt8;
static constexpr bool allow_fixed_string = false;
static inline ResultType apply(A a)
{

View File

@ -10,6 +10,7 @@ template <typename A>
struct BitNotImpl
{
using ResultType = typename NumberTraits::ResultOfBitNot<A>::Type;
static const constexpr bool allow_fixed_string = true;
static inline ResultType apply(A a)
{

View File

@ -8,6 +8,7 @@ template <typename A, typename B>
struct BitOrImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static constexpr const bool allow_fixed_string = true;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
@ -28,7 +29,7 @@ struct BitOrImpl
};
struct NameBitOr { static constexpr auto name = "bitOr"; };
using FunctionBitOr = FunctionBinaryArithmetic<BitOrImpl, NameBitOr>;
using FunctionBitOr = FunctionBinaryArithmetic<BitOrImpl, NameBitOr, true>;
void registerFunctionBitOr(FunctionFactory & factory)
{

View File

@ -8,6 +8,7 @@ template <typename A, typename B>
struct BitRotateLeftImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)

View File

@ -8,6 +8,7 @@ template <typename A, typename B>
struct BitRotateRightImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)

View File

@ -8,6 +8,7 @@ template <typename A, typename B>
struct BitShiftLeftImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)

View File

@ -8,6 +8,7 @@ template <typename A, typename B>
struct BitShiftRightImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)

View File

@ -15,6 +15,7 @@ namespace DB
struct BitSwapLastTwoImpl
{
using ResultType = UInt8;
static constexpr const bool allow_fixed_string = false;
static inline ResultType NO_SANITIZE_UNDEFINED apply(A a)
{

View File

@ -10,6 +10,7 @@ template <typename A, typename B>
struct BitTestImpl
{
using ResultType = UInt8;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
NO_SANITIZE_UNDEFINED static inline Result apply(A a, B b)

View File

@ -16,6 +16,7 @@ namespace DB
struct BitWrapperFuncImpl
{
using ResultType = UInt8;
static constexpr const bool allow_fixed_string = false;
static inline ResultType NO_SANITIZE_UNDEFINED apply(A a)
{

View File

@ -8,6 +8,7 @@ template <typename A, typename B>
struct BitXorImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static constexpr bool allow_fixed_string = true;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
@ -28,7 +29,7 @@ struct BitXorImpl
};
struct NameBitXor { static constexpr auto name = "bitXor"; };
using FunctionBitXor = FunctionBinaryArithmetic<BitXorImpl, NameBitXor>;
using FunctionBitXor = FunctionBinaryArithmetic<BitXorImpl, NameBitXor, true>;
void registerFunctionBitXor(FunctionFactory & factory)
{

View File

@ -72,11 +72,11 @@ public:
for (size_t i = 0; i < input_rows_count; ++i)
{
StringRef source = column_concrete->getDataAt(i);
StringRef source = column_concrete->getDataAtWithTerminatingZero(i);
auto demangled = tryDemangle(source.data);
if (demangled)
{
result_column->insertDataWithTerminatingZero(demangled.get(), strlen(demangled.get()));
result_column->insertDataWithTerminatingZero(demangled.get(), strlen(demangled.get()) + 1);
}
else
{

View File

@ -9,6 +9,7 @@ struct DivideFloatingImpl
{
using ResultType = typename NumberTraits::ResultOfFloatingPointDivision<A, B>::Type;
static const constexpr bool allow_decimal = true;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)

View File

@ -10,6 +10,7 @@ template <typename A, typename B>
struct GCDImpl
{
using ResultType = typename NumberTraits::ResultOfAdditionMultiplication<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -9,6 +9,7 @@ template <typename A, typename B>
struct GreatestBaseImpl
{
using ResultType = NumberTraits::ResultOfGreatest<A, B>;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
@ -34,6 +35,7 @@ template <typename A, typename B>
struct GreatestSpecialImpl
{
using ResultType = std::make_unsigned_t<A>;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -19,6 +19,7 @@ struct DivideIntegralByConstantImpl
: BinaryOperationImplBase<A, B, DivideIntegralImpl<A, B>>
{
using ResultType = typename DivideIntegralImpl<A, B>::ResultType;
static const constexpr bool allow_fixed_string = false;
static void vector_constant(const PaddedPODArray<A> & a, B b, PaddedPODArray<ResultType> & c)
{

View File

@ -50,6 +50,7 @@ template <typename A, typename B>
struct DivideIntegralImpl
{
using ResultType = typename NumberTraits::ResultOfIntegerDivision<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -11,6 +11,7 @@ template <typename A, typename B>
struct DivideIntegralOrZeroImpl
{
using ResultType = typename NumberTraits::ResultOfIntegerDivision<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -10,6 +10,7 @@ template <typename A>
struct IntExp10Impl
{
using ResultType = UInt64;
static constexpr const bool allow_fixed_string = false;
static inline ResultType apply(A a)
{

View File

@ -10,6 +10,7 @@ template <typename A>
struct IntExp2Impl
{
using ResultType = UInt64;
static constexpr const bool allow_fixed_string = false;
static inline ResultType apply(A a)
{

View File

@ -10,6 +10,8 @@ template <typename A, typename B>
struct LCMImpl
{
using ResultType = typename NumberTraits::ResultOfAdditionMultiplication<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -9,6 +9,7 @@ template <typename A, typename B>
struct LeastBaseImpl
{
using ResultType = NumberTraits::ResultOfLeast<A, B>;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
@ -34,6 +35,7 @@ template <typename A, typename B>
struct LeastSpecialImpl
{
using ResultType = std::make_signed_t<A>;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -10,6 +10,7 @@ struct MinusImpl
{
using ResultType = typename NumberTraits::ResultOfSubtraction<A, B>::Type;
static const constexpr bool allow_decimal = true;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)

View File

@ -20,6 +20,7 @@ template <typename A, typename B>
struct ModuloImpl
{
using ResultType = typename NumberTraits::ResultOfModulo<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
@ -38,6 +39,7 @@ struct ModuloByConstantImpl
: BinaryOperationImplBase<A, B, ModuloImpl<A, B>>
{
using ResultType = typename ModuloImpl<A, B>::ResultType;
static const constexpr bool allow_fixed_string = false;
static void vector_constant(const PaddedPODArray<A> & a, B b, PaddedPODArray<ResultType> & c)
{

View File

@ -10,6 +10,7 @@ struct MultiplyImpl
{
using ResultType = typename NumberTraits::ResultOfAdditionMultiplication<A, B>::Type;
static const constexpr bool allow_decimal = true;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)

View File

@ -9,6 +9,7 @@ template <typename A>
struct NegateImpl
{
using ResultType = std::conditional_t<IsDecimalNumber<A>, A, typename NumberTraits::ResultOfNegate<A>::Type>;
static constexpr const bool allow_fixed_string = false;
static inline NO_SANITIZE_UNDEFINED ResultType apply(A a)
{

View File

@ -10,6 +10,7 @@ struct PlusImpl
{
using ResultType = typename NumberTraits::ResultOfAdditionMultiplication<A, B>::Type;
static const constexpr bool allow_decimal = true;
static const constexpr bool allow_fixed_string = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)

View File

@ -8,6 +8,7 @@ template <typename A>
struct RoundAgeImpl
{
using ResultType = UInt8;
static constexpr const bool allow_fixed_string = false;
static inline ResultType apply(A x)
{

View File

@ -8,6 +8,7 @@ template <typename A>
struct RoundDurationImpl
{
using ResultType = UInt16;
static constexpr const bool allow_fixed_string = false;
static inline ResultType apply(A x)
{

View File

@ -49,6 +49,7 @@ template <typename T>
struct RoundToExp2Impl
{
using ResultType = T;
static constexpr const bool allow_fixed_string = false;
static inline T apply(T x)
{

View File

@ -72,7 +72,8 @@ bool Cluster::Address::isLocal(UInt16 clickhouse_port) const
}
Cluster::Address::Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
Cluster::Address::Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, UInt32 shard_index_, UInt32 replica_index_) :
shard_index(shard_index_), replica_index(replica_index_)
{
host_name = config.getString(config_prefix + ".host");
port = static_cast<UInt16>(config.getInt(config_prefix + ".port"));
@ -137,12 +138,8 @@ std::pair<String, UInt16> Cluster::Address::fromString(const String & host_port_
String Cluster::Address::toFullString() const
{
return
escapeForFileName(user) +
(password.empty() ? "" : (':' + escapeForFileName(password))) + '@' +
escapeForFileName(host_name) + ':' +
std::to_string(port) +
(default_database.empty() ? "" : ('#' + escapeForFileName(default_database)))
+ ((secure == Protocol::Secure::Enable) ? "+secure" : "");
((shard_index == 0) ? "" : "shard" + std::to_string(shard_index)) +
((replica_index == 0) ? "" : "_replica" + std::to_string(replica_index));
}
Cluster::Address Cluster::Address::fromFullString(const String & full_string)
@ -150,35 +147,55 @@ Cluster::Address Cluster::Address::fromFullString(const String & full_string)
const char * address_begin = full_string.data();
const char * address_end = address_begin + full_string.size();
Protocol::Secure secure = Protocol::Secure::Disable;
const char * secure_tag = "+secure";
if (endsWith(full_string, secure_tag))
{
address_end -= strlen(secure_tag);
secure = Protocol::Secure::Enable;
}
const char * user_pw_end = strchr(full_string.data(), '@');
const char * colon = strchr(full_string.data(), ':');
if (!user_pw_end || !colon)
throw Exception("Incorrect user[:password]@host:port#default_database format " + full_string, ErrorCodes::SYNTAX_ERROR);
const bool has_pw = colon < user_pw_end;
const char * host_end = has_pw ? strchr(user_pw_end + 1, ':') : colon;
if (!host_end)
throw Exception("Incorrect address '" + full_string + "', it does not contain port", ErrorCodes::SYNTAX_ERROR);
/// parsing with the new [shard{shard_index}[_replica{replica_index}]] format
if (!user_pw_end && startsWith(full_string, "shard"))
{
const char * underscore = strchr(full_string.data(), '_');
const char * has_db = strchr(full_string.data(), '#');
const char * port_end = has_db ? has_db : address_end;
Address address;
address.shard_index = parse<UInt32>(address_begin + strlen("shard"));
address.replica_index = underscore ? parse<UInt32>(underscore + strlen("_replica")) : 0;
Address address;
address.secure = secure;
address.port = parse<UInt16>(host_end + 1, port_end - (host_end + 1));
address.host_name = unescapeForFileName(std::string(user_pw_end + 1, host_end));
address.user = unescapeForFileName(std::string(address_begin, has_pw ? colon : user_pw_end));
address.password = has_pw ? unescapeForFileName(std::string(colon + 1, user_pw_end)) : std::string();
address.default_database = has_db ? unescapeForFileName(std::string(has_db + 1, address_end)) : std::string();
return address;
return address;
}
else
{
/// parsing with the old user[:password]@host:port#default_database format
/// This format is appeared to be inconvenient for the following reasons:
/// - credentials are exposed in file name;
/// - the file name can be too long.
Protocol::Secure secure = Protocol::Secure::Disable;
const char * secure_tag = "+secure";
if (endsWith(full_string, secure_tag))
{
address_end -= strlen(secure_tag);
secure = Protocol::Secure::Enable;
}
const char * colon = strchr(full_string.data(), ':');
if (!user_pw_end || !colon)
throw Exception("Incorrect user[:password]@host:port#default_database format " + full_string, ErrorCodes::SYNTAX_ERROR);
const bool has_pw = colon < user_pw_end;
const char * host_end = has_pw ? strchr(user_pw_end + 1, ':') : colon;
if (!host_end)
throw Exception("Incorrect address '" + full_string + "', it does not contain port", ErrorCodes::SYNTAX_ERROR);
const char * has_db = strchr(full_string.data(), '#');
const char * port_end = has_db ? has_db : address_end;
Address address;
address.secure = secure;
address.port = parse<UInt16>(host_end + 1, port_end - (host_end + 1));
address.host_name = unescapeForFileName(std::string(user_pw_end + 1, host_end));
address.user = unescapeForFileName(std::string(address_begin, has_pw ? colon : user_pw_end));
address.password = has_pw ? unescapeForFileName(std::string(colon + 1, user_pw_end)) : std::string();
address.default_database = has_db ? unescapeForFileName(std::string(has_db + 1, address_end)) : std::string();
return address;
}
}
@ -309,7 +326,7 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting
if (startsWith(replica_key, "replica"))
{
replica_addresses.emplace_back(config, partial_prefix + replica_key);
replica_addresses.emplace_back(config, partial_prefix + replica_key, current_shard_num, current_replica_num);
++current_replica_num;
if (!replica_addresses.back().is_local)

View File

@ -57,6 +57,9 @@ public:
UInt16 port;
String user;
String password;
UInt32 shard_index{}; /// shard serial number in configuration file, starting from 1.
UInt32 replica_index{}; /// replica serial number in this shard, starting from 1; zero means no replicas.
/// This database is selected when no database is specified for Distributed table
String default_database;
/// The locality is determined at the initialization, and is not changed even if DNS is changed
@ -67,7 +70,7 @@ public:
Protocol::Secure secure = Protocol::Secure::Disable;
Address() = default;
Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, UInt32 shard_index_ = 0, UInt32 replica_index_ = 0);
Address(const String & host_port_, const String & user_, const String & password_, UInt16 clickhouse_port, bool secure_ = false);
/// Returns 'escaped_host_name:port'
@ -80,8 +83,10 @@ public:
static std::pair<String, UInt16> fromString(const String & host_port_string);
/// Retrurns escaped user:password@resolved_host_address:resolved_host_port#default_database
/// Returns escaped shard{shard_index}_replica{replica_index}
String toFullString() const;
/// Returns address with only shard index and replica index or full address without shard index and replica index
static Address fromFullString(const String & address_full_string);
/// Returns resolved address if it does resolve.

View File

@ -485,7 +485,7 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat
void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
{
if (create.storage)
if (create.storage || create.is_view || create.is_materialized_view || create.is_live_view || create.is_dictionary)
{
if (create.temporary && create.storage->engine->name != "Memory")
throw Exception(
@ -495,7 +495,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
return;
}
if (create.temporary && !create.is_live_view)
if (create.temporary)
{
auto engine_ast = std::make_shared<ASTFunction>();
engine_ast->name = "Memory";
@ -538,7 +538,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
{
/// Temporary tables are created out of databases.
if (create.temporary && !create.database.empty() && !create.is_live_view)
if (create.temporary && !create.database.empty())
throw Exception("Temporary tables cannot be inside a database. You should not specify a database for a temporary table.",
ErrorCodes::BAD_DATABASE_FOR_TEMPORARY_TABLE);
@ -585,7 +585,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create,
DatabasePtr database;
const String & table_name = create.table;
bool need_add_to_database = !create.temporary || create.is_live_view;
bool need_add_to_database = !create.temporary;
if (need_add_to_database)
{
database = context.getDatabase(create.database);

View File

@ -6,7 +6,6 @@
#include <boost/noncopyable.hpp>
#include <common/logger_useful.h>
#include <Core/Types.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Storages/IStorage.h>
#include <Interpreters/Context.h>
#include <Common/Stopwatch.h>
@ -53,6 +52,10 @@ namespace DB
};
*/
namespace ErrorCodes
{
extern const int TIMEOUT_EXCEEDED;
}
#define DBMS_SYSTEM_LOG_QUEUE_SIZE 1048576
@ -119,55 +122,45 @@ public:
void shutdown();
protected:
Logger * log;
private:
/* Saving thread data */
Context & context;
const String database_name;
const String table_name;
const String storage_def;
StoragePtr table;
bool is_prepared = false;
const size_t flush_interval_milliseconds;
std::atomic<bool> is_shutdown{false};
enum class EntryType
{
LOG_ELEMENT = 0,
AUTO_FLUSH,
FORCE_FLUSH,
SHUTDOWN,
};
using QueueItem = std::pair<EntryType, LogElement>;
/// Queue is bounded. But its size is quite large to not block in all normal cases.
ConcurrentBoundedQueue<QueueItem> queue {DBMS_SYSTEM_LOG_QUEUE_SIZE};
/** Data that was pulled from queue. Data is accumulated here before enough time passed.
* It's possible to implement double-buffering, but we assume that insertion into table is faster
* than accumulation of large amount of log records (for example, for query log - processing of large amount of queries).
*/
std::vector<LogElement> data;
Logger * log;
/** In this thread, data is pulled from 'queue' and stored in 'data', and then written into table.
*/
ThreadFromGlobalPool saving_thread;
void threadFunction();
/* Data shared between callers of add()/flush()/shutdown(), and the saving thread */
std::mutex mutex;
// Queue is bounded. But its size is quite large to not block in all normal cases.
std::vector<LogElement> queue;
// An always-incrementing index of the first message currently in the queue.
// We use it to give a global sequential index to every message, so that we can wait
// until a particular message is flushed. This is used to implement synchronous log
// flushing for SYSTEM FLUSH LOGS.
uint64_t queue_front_index = 0;
bool is_shutdown = false;
std::condition_variable flush_event;
// Requested to flush logs up to this index, exclusive
uint64_t requested_flush_before = 0;
// Flushed log up to this index, exclusive
uint64_t flushed_before = 0;
void savingThreadFunction();
/** Creates new table if it does not exist.
* Renames old table if its structure is not suitable.
* This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created.
*/
bool is_prepared = false;
void prepareTable();
std::mutex flush_mutex;
std::mutex condvar_mutex;
std::condition_variable flush_condvar;
bool force_flushing = false;
/// flushImpl can be executed only in saving_thread.
void flushImpl(EntryType reason);
void flushImpl(const std::vector<LogElement> & to_flush, uint64_t to_flush_end);
};
@ -183,51 +176,92 @@ SystemLog<LogElement>::SystemLog(Context & context_,
{
log = &Logger::get("SystemLog (" + database_name + "." + table_name + ")");
data.reserve(DBMS_SYSTEM_LOG_QUEUE_SIZE);
saving_thread = ThreadFromGlobalPool([this] { threadFunction(); });
saving_thread = ThreadFromGlobalPool([this] { savingThreadFunction(); });
}
template <typename LogElement>
void SystemLog<LogElement>::add(const LogElement & element)
{
std::unique_lock lock(mutex);
if (is_shutdown)
return;
/// Without try we could block here in case of queue overflow.
if (!queue.tryPush({EntryType::LOG_ELEMENT, element}))
LOG_ERROR(log, "SystemLog queue is full");
if (queue.size() == DBMS_SYSTEM_LOG_QUEUE_SIZE / 2)
{
// The queue more than half full, time to flush.
// We only check for strict equality, because messages are added one
// by one, under exclusive lock, so we will see each message count.
// It is enough to only wake the flushing thread once, after the message
// count increases past half available size.
const uint64_t queue_end = queue_front_index + queue.size();
if (requested_flush_before < queue_end)
{
requested_flush_before = queue_end;
}
flush_event.notify_all();
LOG_INFO(log, "Queue is half full for system log '" + demangle(typeid(*this).name()) + "'.");
}
if (queue.size() >= DBMS_SYSTEM_LOG_QUEUE_SIZE)
{
// TextLog sets its logger level to 0, so this log is a noop and there
// is no recursive logging.
LOG_ERROR(log, "Queue is full for system log '" + demangle(typeid(*this).name()) + "'.");
return;
}
queue.push_back(element);
}
template <typename LogElement>
void SystemLog<LogElement>::flush()
{
std::unique_lock lock(mutex);
if (is_shutdown)
return;
std::lock_guard flush_lock(flush_mutex);
force_flushing = true;
const uint64_t queue_end = queue_front_index + queue.size();
/// Tell thread to execute extra flush.
queue.push({EntryType::FORCE_FLUSH, {}});
if (requested_flush_before < queue_end)
{
requested_flush_before = queue_end;
flush_event.notify_all();
}
/// Wait for flush being finished.
std::unique_lock lock(condvar_mutex);
while (force_flushing)
flush_condvar.wait(lock);
// Use an arbitrary timeout to avoid endless waiting.
const int timeout_seconds = 60;
bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds),
[&] { return flushed_before >= queue_end; });
if (!result)
{
throw Exception("Timeout exceeded (" + toString(timeout_seconds) + " s) while flushing system log '" + demangle(typeid(*this).name()) + "'.",
ErrorCodes::TIMEOUT_EXCEEDED);
}
}
template <typename LogElement>
void SystemLog<LogElement>::shutdown()
{
bool old_val = false;
if (!is_shutdown.compare_exchange_strong(old_val, true))
return;
{
std::unique_lock lock(mutex);
if (is_shutdown)
{
return;
}
is_shutdown = true;
/// Tell thread to shutdown.
flush_event.notify_all();
}
/// Tell thread to shutdown.
queue.push({EntryType::SHUTDOWN, {}});
saving_thread.join();
}
@ -240,71 +274,44 @@ SystemLog<LogElement>::~SystemLog()
template <typename LogElement>
void SystemLog<LogElement>::threadFunction()
void SystemLog<LogElement>::savingThreadFunction()
{
setThreadName("SystemLogFlush");
Stopwatch time_after_last_write;
bool first = true;
while (true)
std::vector<LogElement> to_flush;
bool exit_this_thread = false;
while (!exit_this_thread)
{
try
{
if (first)
// The end index (exclusive, like std end()) of the messages we are
// going to flush.
uint64_t to_flush_end = 0;
{
time_after_last_write.restart();
first = false;
std::unique_lock lock(mutex);
flush_event.wait_for(lock, std::chrono::milliseconds(flush_interval_milliseconds),
[&] () { return requested_flush_before > flushed_before || is_shutdown; });
queue_front_index += queue.size();
to_flush_end = queue_front_index;
// Swap with existing array from previous flush, to save memory
// allocations.
to_flush.resize(0);
queue.swap(to_flush);
exit_this_thread = is_shutdown;
}
QueueItem element;
bool has_element = false;
/// data.size() is increased only in this function
/// TODO: get rid of data and queue duality
if (data.empty())
if (to_flush.empty())
{
queue.pop(element);
has_element = true;
}
else
{
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
if (milliseconds_elapsed < flush_interval_milliseconds)
has_element = queue.tryPop(element, flush_interval_milliseconds - milliseconds_elapsed);
continue;
}
if (has_element)
{
if (element.first == EntryType::SHUTDOWN)
{
/// NOTE: MergeTree engine can write data even it is already in shutdown state.
flushImpl(element.first);
break;
}
else if (element.first == EntryType::FORCE_FLUSH)
{
flushImpl(element.first);
time_after_last_write.restart();
continue;
}
else
data.push_back(element.second);
}
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
if (milliseconds_elapsed >= flush_interval_milliseconds)
{
/// Write data to a table.
flushImpl(EntryType::AUTO_FLUSH);
time_after_last_write.restart();
}
flushImpl(to_flush, to_flush_end);
}
catch (...)
{
/// In case of exception we lost accumulated data - to avoid locking.
data.clear();
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
@ -312,13 +319,10 @@ void SystemLog<LogElement>::threadFunction()
template <typename LogElement>
void SystemLog<LogElement>::flushImpl(EntryType reason)
void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush, uint64_t to_flush_end)
{
try
{
if ((reason == EntryType::AUTO_FLUSH || reason == EntryType::SHUTDOWN) && data.empty())
return;
LOG_TRACE(log, "Flushing system log");
/// We check for existence of the table and create it as needed at every flush.
@ -327,13 +331,9 @@ void SystemLog<LogElement>::flushImpl(EntryType reason)
prepareTable();
Block block = LogElement::createBlock();
for (const LogElement & elem : data)
for (const auto & elem : to_flush)
elem.appendToBlock(block);
/// Clear queue early, because insertion to the table could lead to generation of more log entrites
/// and pushing them to already full queue will lead to deadlock.
data.clear();
/// We write to table indirectly, using InterpreterInsertQuery.
/// This is needed to support DEFAULT-columns in table.
@ -352,15 +352,11 @@ void SystemLog<LogElement>::flushImpl(EntryType reason)
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
/// In case of exception, also clean accumulated data - to avoid locking.
data.clear();
}
if (reason == EntryType::FORCE_FLUSH)
{
std::lock_guard lock(condvar_mutex);
force_flushing = false;
flush_condvar.notify_one();
}
std::unique_lock lock(mutex);
flushed_before = to_flush_end;
flush_event.notify_all();
}

View File

@ -70,4 +70,15 @@ void TextLogElement::appendToBlock(Block & block) const
columns[i++]->insert(source_line);
}
TextLog::TextLog(Context & context_, const String & database_name_,
const String & table_name_, const String & storage_def_,
size_t flush_interval_milliseconds_)
: SystemLog<TextLogElement>(context_, database_name_, table_name_,
storage_def_, flush_interval_milliseconds_)
{
// SystemLog methods may write text logs, so we disable logging for the text
// log table to avoid recursion.
log->setLevel(0);
}
}

View File

@ -30,7 +30,13 @@ struct TextLogElement
class TextLog : public SystemLog<TextLogElement>
{
using SystemLog<TextLogElement>::SystemLog;
public:
TextLog(
Context & context_,
const String & database_name_,
const String & table_name_,
const String & storage_def_,
size_t flush_interval_milliseconds_);
};
}

View File

@ -234,6 +234,9 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
<< (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "")
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
if (live_view_timeout)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH TIMEOUT " << (settings.hilite ? hilite_none : "")
<< *live_view_timeout;
formatOnCluster(settings);
}
else

View File

@ -72,6 +72,7 @@ public:
ASTPtr as_table_function;
ASTSelectWithUnionQuery * select = nullptr;
ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.)
std::optional<UInt64> live_view_timeout; /// For CREATE LIVE VIEW ... WITH TIMEOUT ...
/** Get the text that identifies this element. */
String getID(char delim) const override { return (attach ? "AttachQuery" : "CreateQuery") + (delim + database) + delim + table; }

View File

@ -117,7 +117,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
auto index = std::make_shared<ASTIndexDeclaration>();
index->name = name->as<ASTIdentifier &>().name;
index->granularity = granularity->as<ASTLiteral &>().value.get<UInt64>();
index->granularity = granularity->as<ASTLiteral &>().value.safeGet<UInt64>();
index->set(index->expr, expr);
index->set(index->type, type);
node = index;
@ -490,7 +490,6 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_as("AS");
@ -513,11 +512,11 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
ASTPtr as_database;
ASTPtr as_table;
ASTPtr select;
ASTPtr live_view_timeout;
String cluster_str;
bool attach = false;
bool if_not_exists = false;
bool is_temporary = false;
if (!s_create.ignore(pos, expected))
{
@ -527,11 +526,6 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
return false;
}
if (s_temporary.ignore(pos, expected))
{
is_temporary = true;
}
if (!s_live.ignore(pos, expected))
return false;
@ -551,6 +545,12 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
return false;
}
if (ParserKeyword{"WITH TIMEOUT"}.ignore(pos, expected))
{
if (!ParserNumber{}.parse(pos, live_view_timeout, expected))
live_view_timeout = std::make_shared<ASTLiteral>(static_cast<UInt64>(DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC));
}
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
@ -595,7 +595,6 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
query->attach = attach;
query->if_not_exists = if_not_exists;
query->is_live_view = true;
query->temporary = is_temporary;
tryGetIdentifierNameInto(database, query->database);
tryGetIdentifierNameInto(table, query->table);
@ -610,6 +609,9 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
tryGetIdentifierNameInto(as_table, query->as_table);
query->set(query->select, select);
if (live_view_timeout)
query->live_view_timeout.emplace(live_view_timeout->as<ASTLiteral &>().value.safeGet<UInt64>());
return true;
}
@ -672,7 +674,6 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_as("AS");

View File

@ -190,6 +190,12 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
const auto & shards_info = cluster->getShardsInfo();
const auto & shards_addresses = cluster->getShardsAddresses();
/// check new format shard{shard_index}_number{number_index}
if (address.shard_index != 0)
{
return shards_info[address.shard_index - 1].per_replica_pools[address.replica_index - 1];
}
/// existing connections pool have a higher priority
for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index)
{
@ -199,8 +205,15 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
{
const Cluster::Address & replica_address = replicas_addresses[replica_index];
if (address == replica_address)
if (address.user == replica_address.user &&
address.password == replica_address.password &&
address.host_name == replica_address.host_name &&
address.port == replica_address.port &&
address.default_database == replica_address.default_database &&
address.secure == replica_address.secure)
{
return shards_info[shard_index].per_replica_pools[replica_index];
}
}
}

View File

@ -262,8 +262,11 @@ StorageLiveView::StorageLiveView(
global_context.addDependency(select_table_id, table_id_);
is_temporary = query.temporary;
temporary_live_view_timeout = local_context.getSettingsRef().temporary_live_view_timeout.totalSeconds();
if (query.live_view_timeout)
{
is_temporary = true;
temporary_live_view_timeout = *query.live_view_timeout;
}
blocks_ptr = std::make_shared<BlocksPtr>();
blocks_metadata_ptr = std::make_shared<BlocksMetadataPtr>();
@ -566,7 +569,7 @@ BlockInputStreams StorageLiveView::watch(
std::static_pointer_cast<StorageLiveView>(shared_from_this()),
blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit,
context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(),
context.getSettingsRef().temporary_live_view_timeout.totalSeconds());
temporary_live_view_timeout);
{
std::lock_guard no_users_thread_lock(no_users_thread_mutex);
@ -597,7 +600,7 @@ BlockInputStreams StorageLiveView::watch(
std::static_pointer_cast<StorageLiveView>(shared_from_this()),
blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit,
context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(),
context.getSettingsRef().temporary_live_view_timeout.totalSeconds());
temporary_live_view_timeout);
{
std::lock_guard no_users_thread_lock(no_users_thread_mutex);

View File

@ -333,8 +333,8 @@ void MergeTreeRangeReader::ReadResult::optimize()
filter_holder_original = std::move(filter_holder);
filter = new_filter.get();
filter_holder = std::move(new_filter);
need_filter = true;
}
need_filter = true;
}
/// Another guess, if it's worth filtering at PREWHERE
else if (countBytesInResultFilter(filter->getData()) < 0.6 * filter->size())

View File

@ -108,7 +108,18 @@ void StorageMergeTree::shutdown()
if (shutdown_called)
return;
shutdown_called = true;
clearOldPartsFromFilesystem(true);
try
{
clearOldPartsFromFilesystem(true);
}
catch (...)
{
/// Example: the case of readonly filesystem, we have failure removing old parts.
/// Should not prevent table shutdown.
tryLogCurrentException(log);
}
merger_mutator.merges_blocker.cancelForever();
parts_mover.moves_blocker.cancelForever();

View File

@ -1130,6 +1130,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
"6. Data corruption in memory due to hardware issue. "
"7. Manual modification of source data after server startup. "
"8. Manual modification of checksums stored in ZooKeeper. "
"9. Part format related settings like 'enable_mixed_granularity_parts' are different on different replicas. "
"We will download merged part from replica to force byte-identical result.");
write_part_log(ExecutionStatus::fromCurrentException());
@ -4085,37 +4086,42 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
res.replica_path = replica_path;
res.columns_version = columns_version;
if (res.is_session_expired || !with_zk_fields)
res.log_max_index = 0;
res.log_pointer = 0;
res.total_replicas = 0;
res.active_replicas = 0;
if (with_zk_fields && !res.is_session_expired)
{
res.log_max_index = 0;
res.log_pointer = 0;
res.total_replicas = 0;
res.active_replicas = 0;
}
else
{
auto log_entries = zookeeper->getChildren(zookeeper_path + "/log");
if (log_entries.empty())
try
{
res.log_max_index = 0;
auto log_entries = zookeeper->getChildren(zookeeper_path + "/log");
if (log_entries.empty())
{
res.log_max_index = 0;
}
else
{
const String & last_log_entry = *std::max_element(log_entries.begin(), log_entries.end());
res.log_max_index = parse<UInt64>(last_log_entry.substr(strlen("log-")));
}
String log_pointer_str = zookeeper->get(replica_path + "/log_pointer");
res.log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);
auto all_replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
res.total_replicas = all_replicas.size();
res.active_replicas = 0;
for (const String & replica : all_replicas)
if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
++res.active_replicas;
}
else
catch (const Coordination::Exception &)
{
const String & last_log_entry = *std::max_element(log_entries.begin(), log_entries.end());
res.log_max_index = parse<UInt64>(last_log_entry.substr(strlen("log-")));
res.zookeeper_exception = getCurrentExceptionMessage(false);
}
String log_pointer_str = zookeeper->get(replica_path + "/log_pointer");
res.log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);
auto all_replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
res.total_replicas = all_replicas.size();
res.active_replicas = 0;
for (const String & replica : all_replicas)
if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
++res.active_replicas;
}
}

View File

@ -150,6 +150,8 @@ public:
UInt64 absolute_delay;
UInt8 total_replicas;
UInt8 active_replicas;
/// If the error has happened fetching the info from ZooKeeper, this field will be set.
String zookeeper_exception;
};
/// Get the status of the table. If with_zk_fields = false - do not fill in the fields that require queries to ZK.

View File

@ -50,6 +50,7 @@ StorageSystemReplicas::StorageSystemReplicas(const std::string & name_)
{ "absolute_delay", std::make_shared<DataTypeUInt64>() },
{ "total_replicas", std::make_shared<DataTypeUInt8>() },
{ "active_replicas", std::make_shared<DataTypeUInt8>() },
{ "zookeeper_exception", std::make_shared<DataTypeString>() },
}));
}
@ -93,7 +94,8 @@ Pipes StorageSystemReplicas::readWithProcessors(
if ( column_name == "log_max_index"
|| column_name == "log_pointer"
|| column_name == "total_replicas"
|| column_name == "active_replicas")
|| column_name == "active_replicas"
|| column_name == "zookeeper_exception")
{
with_zk_fields = true;
break;
@ -175,6 +177,7 @@ Pipes StorageSystemReplicas::readWithProcessors(
res_columns[col_num++]->insert(status.absolute_delay);
res_columns[col_num++]->insert(status.total_replicas);
res_columns[col_num++]->insert(status.active_replicas);
res_columns[col_num++]->insert(status.zookeeper_exception);
}
Block header = getSampleBlock();

View File

@ -88,6 +88,9 @@ try
&& !endsWith(it->path, "/columns")
&& !endsWith(it->path, "/checksums"))
{
/// The node is related to part.
/// If it is the part in old format (the node contains children) - convert it to the new format.
if (!children_response.names.empty())
{
auto part_header = DB::ReplicatedMergeTreePartHeader::fromColumnsAndChecksumsZNodes(
@ -103,6 +106,7 @@ try
}
else
{
/// Recursively add children to the queue.
for (const auto & name : children_response.names)
{
std::string child_path = it->path == "/" ? it->path + name : it->path + '/' + name;

View File

@ -0,0 +1,9 @@
<yandex>
<storage_configuration>
<disks>
<disk_memory>
<type>memory</type>
</disk_memory>
</disks>
</storage_configuration>
</yandex>

View File

@ -26,12 +26,12 @@ def test_single_file(started_cluster):
node.query("create table distr_1 (x UInt64, s String) engine = Distributed('test_cluster', database, table)")
node.query("insert into distr_1 values (1, 'a'), (2, 'bb'), (3, 'ccc')")
query = "select * from file('/var/lib/clickhouse/data/default/distr_1/default@not_existing:9000/1.bin', 'Distributed')"
query = "select * from file('/var/lib/clickhouse/data/default/distr_1/shard1_replica1/1.bin', 'Distributed')"
out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query])
assert out == '1\ta\n2\tbb\n3\tccc\n'
query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/default/distr_1/default@not_existing:9000/1.bin');" \
query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/default/distr_1/shard1_replica1/1.bin');" \
"select * from t"
out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query])
@ -45,12 +45,12 @@ def test_two_files(started_cluster):
node.query("insert into distr_2 values (0, '_'), (1, 'a')")
node.query("insert into distr_2 values (2, 'bb'), (3, 'ccc')")
query = "select * from file('/var/lib/clickhouse/data/default/distr_2/default@not_existing:9000/{1,2,3,4}.bin', 'Distributed') order by x"
query = "select * from file('/var/lib/clickhouse/data/default/distr_2/shard1_replica1/{1,2,3,4}.bin', 'Distributed') order by x"
out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query])
assert out == '0\t_\n1\ta\n2\tbb\n3\tccc\n'
query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/default/distr_2/default@not_existing:9000/{1,2,3,4}.bin');" \
query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/default/distr_2/shard1_replica1/{1,2,3,4}.bin');" \
"select * from t order by x"
out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query])

View File

@ -25,7 +25,7 @@ def _files_in_dist_mon(node, root, table):
'bash',
'-c',
# `-maxdepth 1` to avoid /tmp/ subdirectory
'find /{root}/data/default/{table}/default@127%2E0%2E0%2E2:9000 -maxdepth 1 -type f | wc -l'.format(root=root, table=table)
'find /{root}/data/default/{table}/shard2_replica1 -maxdepth 1 -type f | wc -l'.format(root=root, table=table)
]).split('\n')[0])
def test_different_versions(start_cluster):

View File

@ -0,0 +1,24 @@
<test>
<type>loop</type>
<stop_conditions>
<all_of>
<iterations>3</iterations>
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
</all_of>
<any_of>
<iterations>5</iterations>
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<preconditions>
<table_exists>test.hits</table_exists>
</preconditions>
<query>SELECT count() FROM test.hits WHERE NOT ignore(bitAnd(toFixedString(ClientIP6, 16), IPv6StringToNum('ffff:ffff:ffff:0000:0000:0000:0000:0000')))</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(bitOr(toFixedString(ClientIP6, 16), IPv6StringToNum('ffff:ffff:ffff:0000:0000:0000:0000:0000')))</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(bitXor(toFixedString(ClientIP6, 16), IPv6StringToNum('ffff:ffff:ffff:0000:0000:0000:0000:0000')))</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(bitNot(toFixedString(ClientIP6, 16)))</query>
</test>

View File

@ -10,7 +10,7 @@
<create_query>DROP TABLE IF EXISTS test_bf</create_query>
<create_query>CREATE TABLE test_bf (`id` int, `ary` Array(String), INDEX idx_ary ary TYPE bloom_filter(0.01) GRANULARITY 8192) ENGINE = MergeTree() ORDER BY id</create_query>
<create_query>SYSTEM STOP MERGES</create_query>
<query>INSERT INTO test_bf SELECT number AS id, [CAST(id, 'String'), CAST(id + 1, 'String'), CAST(id + 2, 'String')] FROM system.numbers LIMIT 3000000</query>
<query>INSERT INTO test_bf SELECT number AS id, [CAST(id, 'String'), CAST(id + 1, 'String'), CAST(id + 2, 'String')] FROM numbers(1000000)</query>
<drop_query>SYSTEM START MERGES</drop_query>
<drop_query>DROP TABLE IF EXISTS test_bf</drop_query>
</test>

View File

@ -42,18 +42,22 @@
<substitution>
<name>num_rows</name>
<values>
<value>1000000</value>
<value>10000000</value>
</values>
</substitution>
</substitutions>
<create_query>CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple();</create_query>
<create_query>CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec}))
ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple()
SETTINGS parts_to_delay_insert = 5000, parts_to_throw_insert = 5000;</create_query>
<create_query>system stop merges</create_query>
<!-- Using limit to make query finite, allowing it to be run multiple times in a loop, reducing mean error -->
<query>INSERT INTO codec_seq_{type}_{codec} (n) SELECT number FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</query>
<query>INSERT INTO codec_mon_{type}_{codec} (n) SELECT number*512+(intHash64(number)%512) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</query>
<query>INSERT INTO codec_rnd_{type}_{codec} (n) SELECT intHash64(number) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</query>
<drop_query>system start merges</drop_query>
<drop_query>DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec}</drop_query>
</test>

View File

@ -14,7 +14,8 @@
<preconditions>
<table_exists>test.hits</table_exists>
<table_exists>hits_100m_single</table_exists>
<table_exists>hits_10m_single</table_exists>
</preconditions>
<!-- 14. Некриптографическая хэш-функция для строк небольшой длины.
@ -22,76 +23,76 @@
Строка SearchPhrase непустая только в 6.8% случаев. Средняя длина непустой строки - 47 байт.
URL почти всегда непуст и его средняя длина - 77 байт.
PageCharset тоже почти всегда непуст, но его средняя длина поменьше - 6.2 байта. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(cityHash64(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(cityHash64(SearchPhrase))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(cityHash64(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(cityHash64(SearchPhrase))</query>
<!-- 15. Некриптографическая хэш-функция для строк небольшой длины. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(farmHash64(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(farmHash64(SearchPhrase))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(farmHash64(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(farmHash64(SearchPhrase))</query>
<!-- 16. Некриптографическая хэш-функция для строк небольшой длины. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(metroHash64(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(metroHash64(SearchPhrase))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(metroHash64(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(metroHash64(SearchPhrase))</query>
<!-- 17. Криптографическая хэш-функция для строк. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(sipHash64(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(sipHash64(SearchPhrase))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(sipHash64(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(sipHash64(SearchPhrase))</query>
<!-- 18. Криптографическая хэш-функция для строк. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(MD5(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(MD5(SearchPhrase))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(MD5(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(MD5(SearchPhrase))</query>
<!-- 19. Криптографическая хэш-функция для строк. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(MD5(URL)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(MD5(URL))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(MD5(URL)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(MD5(URL))</query>
<!-- 20. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(cityHash64(URL)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(cityHash64(URL))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(cityHash64(URL)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(cityHash64(URL))</query>
<!-- 21. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(sipHash64(URL)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(sipHash64(URL))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(sipHash64(URL)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(sipHash64(URL))</query>
<!-- 22. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(cityHash64(PageCharset)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(cityHash64(PageCharset))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(cityHash64(PageCharset)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(cityHash64(PageCharset))</query>
<!-- 23. Поиск подстроки в строке. -->
<query>SELECT count() FROM test.hits WHERE URL LIKE '%metrika%' SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE URL LIKE '%metrika%'</query>
<query>SELECT count() FROM hits_10m_single WHERE URL LIKE '%metrika%' SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE URL LIKE '%metrika%'</query>
<!-- 24. Более сложный поиск подстроки в строке. -->
<query>SELECT count() FROM test.hits WHERE positionCaseInsensitiveUTF8(URL, 'новости') != 0 SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE positionCaseInsensitiveUTF8(URL, 'новости') != 0</query>
<query>SELECT count() FROM hits_10m_single WHERE positionCaseInsensitiveUTF8(URL, 'новости') != 0 SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE positionCaseInsensitiveUTF8(URL, 'новости') != 0</query>
<!-- 25. Регексп. -->
<query>SELECT count() FROM test.hits WHERE match(URL, '^https?://(?:www\\.)?metri[kc]a\\.yandex\\.(?:ru|com|com\\.tr|ua|by|kz)/.+?2014') SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE match(URL, '^https?://(?:www\\.)?metri[kc]a\\.yandex\\.(?:ru|com|com\\.tr|ua|by|kz)/.+?2014')</query>
<query>SELECT count() FROM hits_10m_single WHERE match(URL, '^https?://(?:www\\.)?metri[kc]a\\.yandex\\.(?:ru|com|com\\.tr|ua|by|kz)/.+?2014') SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE match(URL, '^https?://(?:www\\.)?metri[kc]a\\.yandex\\.(?:ru|com|com\\.tr|ua|by|kz)/.+?2014')</query>
<!-- 34. Сложная агрегация. -->
<query>SELECT SearchEngineID, SearchPhrase, RegionID FROM test.hits GROUP BY SearchEngineID, SearchPhrase, RegionID ORDER BY count() DESC LIMIT 10 SETTINGS max_threads = 1</query>
<query>SELECT SearchEngineID, SearchPhrase, RegionID FROM test.hits GROUP BY SearchEngineID, SearchPhrase, RegionID ORDER BY count() DESC LIMIT 10</query>
<query>SELECT SearchEngineID, SearchPhrase, RegionID FROM hits_10m_single GROUP BY SearchEngineID, SearchPhrase, RegionID ORDER BY count() DESC LIMIT 10 SETTINGS max_threads = 1</query>
<query>SELECT SearchEngineID, SearchPhrase, RegionID FROM hits_100m_single GROUP BY SearchEngineID, SearchPhrase, RegionID ORDER BY count() DESC LIMIT 10</query>
<!-- 36. Функция для работы с датой и временем. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(toMonday(EventTime)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toMonday(EventTime))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(toMonday(EventTime)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(toMonday(EventTime))</query>
<!-- 37. Функция для работы с URL. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(cutQueryString(URL)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(cutQueryString(URL))</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(cutQueryString(URL)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(cutQueryString(URL))</query>
<!-- 38. Разные алгоритмы вычисления квантилей. -->
<query>SELECT quantilesIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM test.hits SETTINGS max_threads = 1</query>
<query>SELECT quantilesIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM test.hits</query>
<query>SELECT quantilesIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_10m_single SETTINGS max_threads = 1</query>
<query>SELECT quantilesIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_100m_single</query>
<!-- 39. Разные алгоритмы вычисления квантилей. -->
<query>SELECT quantilesTimingIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM test.hits SETTINGS max_threads = 1</query>
<query>SELECT quantilesTimingIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM test.hits</query>
<query>SELECT quantilesTimingIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_10m_single SETTINGS max_threads = 1</query>
<query>SELECT quantilesTimingIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_100m_single</query>
<!-- 40. Разные алгоритмы вычисления квантилей. -->
<query>SELECT quantilesExactIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM test.hits SETTINGS max_threads = 1</query>
<query>SELECT quantilesExactIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM test.hits</query>
<query>SELECT quantilesExactIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_10m_single SETTINGS max_threads = 1</query>
<query>SELECT quantilesExactIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_100m_single</query>
<!-- 41. Разные алгоритмы вычисления квантилей. -->
<query>SELECT quantilesTDigestIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM test.hits SETTINGS max_threads = 1</query>
<query>SELECT quantilesTDigestIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM test.hits</query>
<query>SELECT quantilesTDigestIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_10m_single SETTINGS max_threads = 1</query>
<query>SELECT quantilesTDigestIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_100m_single</query>
<!-- 42. Разные алгоритмы вычисления кардинальности. -->
<query>SELECT uniq(UserID) FROM test.hits SETTINGS max_threads = 1</query>
<query>SELECT uniq(UserID) FROM test.hits</query>
<query>SELECT uniq(UserID) FROM hits_10m_single SETTINGS max_threads = 1</query>
<query>SELECT uniq(UserID) FROM hits_100m_single</query>
<!-- 43. Разные алгоритмы вычисления кардинальности. -->
<query>SELECT uniqCombined(17)(UserID) FROM test.hits SETTINGS max_threads = 1</query>
<query>SELECT uniqCombined(17)(UserID) FROM test.hits</query>
<query>SELECT uniqCombined(17)(UserID) FROM hits_10m_single SETTINGS max_threads = 1</query>
<query>SELECT uniqCombined(17)(UserID) FROM hits_100m_single</query>
<!-- 44. Разные алгоритмы вычисления кардинальности. -->
<query>SELECT uniqExact(UserID) FROM test.hits SETTINGS max_threads = 1</query>
<query>SELECT uniqExact(UserID) FROM test.hits</query>
<query>SELECT uniqExact(UserID) FROM hits_10m_single SETTINGS max_threads = 1</query>
<query>SELECT uniqExact(UserID) FROM hits_100m_single</query>
<!-- 45. Что-то чуть более похожее на реальный запрос. -->
<query>SELECT RegionID, uniq(UserID) FROM test.hits GROUP BY RegionID SETTINGS max_threads = 1</query>
<query>SELECT RegionID, uniq(UserID) FROM test.hits GROUP BY RegionID</query>
<query>SELECT RegionID, uniq(UserID) FROM hits_10m_single GROUP BY RegionID SETTINGS max_threads = 1</query>
<query>SELECT RegionID, uniq(UserID) FROM hits_100m_single GROUP BY RegionID</query>
<!-- 47. Читаем и разжимаем все столбцы, и ничего с ними потом не делаем. -->
<query>SELECT count() FROM test.hits WHERE NOT ignore(*) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(*)</query>
<query>SELECT count() FROM hits_10m_single WHERE NOT ignore(*) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore(*)</query>
</test>

View File

@ -33,13 +33,25 @@
<substitution>
<name>value</name>
<values>
<value>toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0')</value>
<value>toDecimal128(number, 23)</value>
<value>materialize(toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0'))</value>
<value>generateUUIDv4()</value>
</values>
</substitution>
<substitution>
<name>value_slow</name>
<values>
<value>toDecimal128(number, 23)</value>
</values>
</substitution>
<substitution>
<name>table</name>
<values>
<value>numbers(100000000)</value>
<value>numbers_mt(1000000000)</value>
</values>
</substitution>
<substitution>
<name>table_slow</name>
<values>
<value>numbers(1000000)</value>
<value>numbers_mt(10000000)</value>
@ -48,4 +60,5 @@
</substitutions>
<query>SELECT count() from {table} where not ignore({hash_func}({value}))</query>
<query>SELECT count() from {table_slow} where not ignore({hash_func}({value_slow}))</query>
</test>

View File

@ -72,14 +72,17 @@
intDiv( number, 4 ) as rowid,
groupArray( base64Encode( left( reinterpretAsString( rand64() ), 6) ) ) as arr
FROM
numbers(10000000)
numbers(1000000)
GROUP BY rowid
) GROUP BY id</fill_query>
<fill_query>optimize table lot_of_string_arrays_src</fill_query>
<fill_query>system stop merges</fill_query>
<query>INSERT INTO lot_of_string_arrays_dst SELECT rand64() as id, columns('^col') FROM lot_of_string_arrays_src</query>
<query>INSERT INTO lot_of_string_arrays_dst_lowcardinality SELECT rand64() as id, columns('^col') FROM lot_of_string_arrays_src</query>
<drop_query>system start merges</drop_query>
<drop_query>DROP TABLE IF EXISTS lot_of_string_arrays_src</drop_query>
<drop_query>DROP TABLE IF EXISTS lot_of_string_arrays_dst</drop_query>
<drop_query>DROP TABLE IF EXISTS lot_of_string_arrays_dst_lowcardinality</drop_query>
</test>
</test>

View File

@ -5,7 +5,7 @@
</tags>
<preconditions>
<table_exists>hits_100m_single</table_exists>
<table_exists>hits_10m_single</table_exists>
</preconditions>
<type>loop</type>
@ -25,22 +25,22 @@
</any_of>
</stop_conditions>
<query>SELECT DISTINCT URL,Title, ngramDistance(Title, URL) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT SearchPhrase,Title, ngramDistance(Title, SearchPhrase) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistance(Title, 'what is love') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistance(Title, 'baby dont hurt me') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistance(Title, 'no more') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'wHAt Is lovE') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'BABY DonT hUrT me') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'nO MOrE') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT URL,Title, ngramDistanceUTF8(Title, URL) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT SearchPhrase,Title, ngramDistanceUTF8(Title, SearchPhrase) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistanceUTF8(Title, 'метрика') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT URL, ngramDistanceUTF8(URL, 'как дела') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT URL, ngramDistanceUTF8(URL, 'чем занимаешься') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT URL,Title, ngramDistance(Title, URL) AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT SearchPhrase,Title, ngramDistance(Title, SearchPhrase) AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT Title, ngramDistance(Title, 'what is love') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT Title, ngramDistance(Title, 'baby dont hurt me') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT Title, ngramDistance(Title, 'no more') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'wHAt Is lovE') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'BABY DonT hUrT me') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'nO MOrE') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT URL,Title, ngramDistanceUTF8(Title, URL) AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT SearchPhrase,Title, ngramDistanceUTF8(Title, SearchPhrase) AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT Title, ngramDistanceUTF8(Title, 'метрика') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT URL, ngramDistanceUTF8(URL, 'как дела') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT URL, ngramDistanceUTF8(URL, 'чем занимаешься') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitiveUTF8(Title, 'Метрика') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT URL, ngramDistanceCaseInsensitiveUTF8(URL, 'как дЕлА') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT URL, ngramDistanceCaseInsensitiveUTF8(URL, 'Чем зАнимаешЬся') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50</query>
<query>SELECT DISTINCT Title, ngramDistanceCaseInsensitiveUTF8(Title, 'Метрика') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT URL, ngramDistanceCaseInsensitiveUTF8(URL, 'как дЕлА') AS distance FROM hits_10m_single FORMAT Null</query>
<query>SELECT DISTINCT URL, ngramDistanceCaseInsensitiveUTF8(URL, 'Чем зАнимаешЬся') AS distance FROM hits_10m_single FORMAT Null</query>
</test>

View File

@ -2,6 +2,7 @@
<type>loop</type>
<preconditions>
<table_exists>hits_10m_single</table_exists>
<table_exists>hits_100m_single</table_exists>
</preconditions>
@ -17,9 +18,9 @@
<query>SELECT count() FROM hits_100m_single WHERE UserID IN (SELECT UserID FROM hits_100m_single WHERE AdvEngineID != 0)</query>
<query>SELECT count() FROM hits_100m_single WHERE UserID IN (SELECT UserID FROM hits_100m_single)</query>
<query>SELECT count() FROM hits_100m_single WHERE SearchPhrase IN (SELECT SearchPhrase FROM hits_100m_single)</query>
<query>SELECT count() FROM hits_100m_single WHERE URL IN (SELECT URL FROM hits_100m_single WHERE AdvEngineID != 0)</query>
<query>SELECT count() FROM hits_100m_single WHERE URL IN (SELECT URL FROM hits_100m_single WHERE SearchEngineID != 0)</query>
<query>SELECT count() FROM hits_100m_single WHERE RegionID IN (SELECT RegionID FROM hits_100m_single)</query>
<query>SELECT count() FROM hits_10m_single WHERE UserID IN (SELECT UserID FROM hits_10m_single)</query>
<query>SELECT count() FROM hits_10m_single WHERE SearchPhrase IN (SELECT SearchPhrase FROM hits_10m_single)</query>
<query>SELECT count() FROM hits_10m_single WHERE URL IN (SELECT URL FROM hits_10m_single WHERE AdvEngineID != 0)</query>
<query>SELECT count() FROM hits_10m_single WHERE URL IN (SELECT URL FROM hits_10m_single WHERE SearchEngineID != 0)</query>
<query>SELECT count() FROM hits_10m_single WHERE RegionID IN (SELECT RegionID FROM hits_10m_single)</query>
</test>

View File

@ -15,8 +15,8 @@
<substitution>
<name>scale</name>
<values>
<value>100000</value>
<value>1000000</value>
<value>10000000</value>
</values>
</substitution>
<substitution>

View File

@ -4,7 +4,7 @@ DROP TABLE IF EXISTS lv;
DROP TABLE IF EXISTS mt;
CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple();
CREATE TEMPORARY LIVE VIEW lv AS SELECT sum(a) FROM mt;
CREATE LIVE VIEW lv WITH TIMEOUT AS SELECT sum(a) FROM mt;
WATCH lv LIMIT 0;

View File

@ -27,7 +27,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo
client1.expect(prompt)
client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()')
client1.expect(prompt)
client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt')
client1.send('CREATE LIVE VIEW test.lv WITH TIMEOUT AS SELECT sum(a) FROM test.mt')
client1.expect(prompt)
client1.send('WATCH test.lv')
client1.expect(r'0.*1' + end_of_block)

View File

@ -25,11 +25,9 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo
client1.expect(prompt)
client1.send('DROP TABLE IF EXISTS test.mt')
client1.expect(prompt)
client1.send('SET temporary_live_view_timeout=1')
client1.expect(prompt)
client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()')
client1.expect(prompt)
client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt')
client1.send('CREATE LIVE VIEW test.lv WITH TIMEOUT 1 AS SELECT sum(a) FROM test.mt')
client1.expect(prompt)
client1.send('WATCH test.lv')
client1.expect(r'0.*1' + end_of_block)

View File

@ -29,7 +29,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo
client1.expect(prompt)
client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()')
client1.expect(prompt)
client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt')
client1.send('CREATE LIVE VIEW test.lv WITH TIMEOUT AS SELECT sum(a) FROM test.mt')
client1.expect(prompt)
client1.send('WATCH test.lv EVENTS')
client2.send('INSERT INTO test.mt VALUES (1),(2),(3)')

View File

@ -29,7 +29,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo
client1.expect(prompt)
client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()')
client1.expect(prompt)
client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt')
client1.send('CREATE LIVE VIEW test.lv WITH TIMEOUT AS SELECT sum(a) FROM test.mt')
client1.expect(prompt)
client1.send('WATCH test.lv')
client1.expect(r'0.*1' + end_of_block)

View File

@ -6,9 +6,8 @@ DROP TABLE IF EXISTS test.mt;
SELECT name, value from system.settings WHERE name = 'temporary_live_view_timeout';
SELECT name, value from system.settings WHERE name = 'live_view_heartbeat_interval';
SET temporary_live_view_timeout=1;
CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple();
CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt;
CREATE LIVE VIEW test.lv WITH TIMEOUT 1 AS SELECT sum(a) FROM test.mt;
SHOW TABLES FROM test LIKE 'lv';
SELECT sleep(2);

View File

@ -44,7 +44,7 @@ def test():
send_query('DROP TABLE IF EXISTS test.lv').read()
send_query('DROP TABLE IF EXISTS test.mt').read()
send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read()
send_query('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read()
send_query('CREATE LIVE VIEW test.lv WITH TIMEOUT AS SELECT sum(a) FROM test.mt').read()
q = queue.Queue()
p = send_query_in_process_group('WATCH test.lv')

View File

@ -44,7 +44,7 @@ def test():
send_query('DROP TABLE IF EXISTS test.lv').read()
send_query('DROP TABLE IF EXISTS test.mt').read()
send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read()
send_query('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read()
send_query('CREATE LIVE VIEW test.lv WITH TIMEOUT AS SELECT sum(a) FROM test.mt').read()
q = queue.Queue()
p = send_query_in_process_group('WATCH test.lv')

View File

@ -1,20 +1,20 @@
0 0 1 2
1 2 0 0
0 0 1 2
-
0 0 1 2
-
0 0 1 2
1 2 0 0
0 0 1 2
-
0 0 1 2
-
1
0
1
-
0
-
1
0
1
-
0
-
@ -48,13 +48,13 @@
-
\N
-
1
\N
1
-
1
-
1
\N
1
-
1
-

View File

@ -1,63 +1,63 @@
SET join_use_nulls = 0;
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY foo.a, foo.b, bar.a, bar.b;
SELECT '-';
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY foo.a, foo.b, bar.a, bar.b;
SELECT '-';
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY foo.a, foo.b, bar.a, bar.b;
SELECT '-';
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY foo.a, foo.b, bar.a, bar.b;
SELECT '-';
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY foo.a;
SELECT '-';
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY foo.a;
SELECT '-';
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY foo.a;
SELECT '-';
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY foo.a;
SELECT '-';
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY bar.a;
SELECT '-';
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY bar.a;
SELECT '-';
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY bar.a;
SELECT '-';
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY bar.a;
SELECT '-';
SET join_use_nulls = 1;
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY foo.a, foo.b, bar.a, bar.b;
SELECT '-';
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY foo.a, foo.b, bar.a, bar.b;
SELECT '-';
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY foo.a, foo.b, bar.a, bar.b;
SELECT '-';
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT * FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY foo.a, foo.b, bar.a, bar.b;
SELECT '-';
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY foo.a;
SELECT '-';
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY foo.a;
SELECT '-';
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY foo.a;
SELECT '-';
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT foo.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY foo.a;
SELECT '-';
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY bar.a;
SELECT '-';
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b);
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.a = bar.b) AND (foo.b = bar.b) ORDER BY bar.a;
SELECT '-';
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo FULL JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY bar.a;
SELECT '-';
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b);
SELECT bar.a FROM (SELECT 1 AS a, 2 AS b) AS foo RIGHT JOIN (SELECT 1 AS a, 2 AS b) AS bar ON (foo.b = bar.a) AND (foo.b = bar.b) ORDER BY bar.a;
SELECT '-';

View File

@ -0,0 +1,130 @@
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2001:db8:85a3::
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
2ff1:db8:85a3:8d3a:b2da:8a2e:370:7334
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
de81:db8:85a3:8d3a:b325:740d:4617:fa3e
dffe:f247:7a5c:72c5:4d25:75d1:fc8f:8ccb
dffe:f247:7a5c:72c5:4d25:75d1:fc8f:8ccb
dffe:f247:7a5c:72c5:4d25:75d1:fc8f:8ccb
dffe:f247:7a5c:72c5:4d25:75d1:fc8f:8ccb
dffe:f247:7a5c:72c5:4d25:75d1:fc8f:8ccb
dffe:f247:7a5c:72c5:4d25:75d1:fc8f:8ccb
dffe:f247:7a5c:72c5:4d25:75d1:fc8f:8ccb
dffe:f247:7a5c:72c5:4d25:75d1:fc8f:8ccb
dffe:f247:7a5c:72c5:4d25:75d1:fc8f:8ccb
dffe:f247:7a5c:72c5:4d25:75d1:fc8f:8ccb

View File

@ -0,0 +1,16 @@
SELECT IPv6NumToString(bitAnd(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334'), IPv6StringToNum('ffff:ffff:ffff:0000:0000:0000:0000:0000'))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitAnd(materialize(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334')), IPv6StringToNum('ffff:ffff:ffff:0000:0000:0000:0000:0000'))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitAnd(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334'), materialize(IPv6StringToNum('ffff:ffff:ffff:0000:0000:0000:0000:0000')))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitAnd(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334'), materialize(IPv6StringToNum('ffff:ffff:ffff:0000:0000:0000:0000:0000')))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitOr(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334'), IPv6StringToNum('2ff0:0000:0000:0000:0000:0000:0000:0000'))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitOr(materialize(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334')), IPv6StringToNum('2ff0:0000:0000:0000:0000:0000:0000:0000'))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitOr(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334'), materialize(IPv6StringToNum('2ff0:0000:0000:0000:0000:0000:0000:0000')))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitOr(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334'), materialize(IPv6StringToNum('2ff0:0000:0000:0000:0000:0000:0000:0000')))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitXor(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334'), IPv6StringToNum('fe80::1ff:fe23:4567:890a'))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitXor(materialize(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334')), IPv6StringToNum('fe80::1ff:fe23:4567:890a'))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitXor(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334'), materialize(IPv6StringToNum('fe80::1ff:fe23:4567:890a')))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitXor(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334'), materialize(IPv6StringToNum('fe80::1ff:fe23:4567:890a')))) FROM system.numbers LIMIT 10;
SELECT IPv6NumToString(bitNot(IPv6StringToNum('2001:0db8:85a3:8d3a:b2da:8a2e:0370:7334'))) FROM system.numbers LIMIT 10;

View File

@ -0,0 +1,11 @@
drop table if exists t;
create table t (a Int) engine = MergeTree order by a;
-- some magic to satisfy conditions to run optimizations in MergeTreeRangeReader
insert into t select number < 20 ? 0 : 1 from numbers(50);
alter table t add column s String default 'foo';
select s from t prewhere a != 1 where rand() % 2 = 0 limit 1;
drop table t;

View File

@ -0,0 +1 @@
FixedString(4)

View File

@ -0,0 +1 @@
select toTypeName(reinterpretAsFixedString(0xdeadbeef));

View File

@ -0,0 +1,13 @@
test1
test1
test1
test1

View File

@ -0,0 +1,32 @@
drop table if exists default.test_table_01080;
CREATE TABLE default.test_table_01080 (dim_key Int64, dim_id String) ENGINE = MergeTree Order by (dim_key);
insert into default.test_table_01080 values(1,'test1');
drop DICTIONARY if exists default.test_dict_01080;
CREATE DICTIONARY default.test_dict_01080 ( dim_key Int64, dim_id String )
PRIMARY KEY dim_key
source(clickhouse(host 'localhost' port '9000' user 'default' password '' db 'default' table 'test_table_01080'))
LIFETIME(MIN 0 MAX 0) LAYOUT(complex_key_hashed());
SELECT dictGetString('default.test_dict_01080', 'dim_id', tuple(toInt64(1)));
SELECT dictGetString('default.test_dict_01080', 'dim_id', tuple(toInt64(0)));
select dictGetString('default.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(0)) as x);
select dictGetString('default.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(1)) as x);
select dictGetString('default.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(number)) as x from numbers(5));
select dictGetString('default.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(rand64()*0)) as x);
select dictGetString('default.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(blockSize()=0)) as x);
select dictGetString('default.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(materialize(0))) as x);
select dictGetString('default.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(materialize(1))) as x);
drop DICTIONARY default.test_dict_01080;
drop table default.test_table_01080;

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