Merge branch 'master' into stress-thread-fuzzer

This commit is contained in:
mergify[bot] 2021-08-21 19:44:23 +00:00 committed by GitHub
commit 14e9209ee2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
58 changed files with 612 additions and 318 deletions

View File

@ -173,6 +173,9 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ
cmake_flags.append('-DUSE_GTEST=1')
cmake_flags.append('-DENABLE_TESTS=1')
cmake_flags.append('-DENABLE_EXAMPLES=1')
cmake_flags.append('-DENABLE_FUZZING=1')
# For fuzzing needs
cmake_flags.append('-DUSE_YAML_CPP=1')
# Don't stop on first error to find more clang-tidy errors in one run.
result.append('NINJA_FLAGS=-k0')

View File

@ -303,6 +303,7 @@ function run_tests
01683_codec_encrypted # Depends on OpenSSL
01776_decrypt_aead_size_check # Depends on OpenSSL
01811_filter_by_null # Depends on OpenSSL
02012_sha512_fixedstring # Depends on OpenSSL
01281_unsucceeded_insert_select_queries_counter
01292_create_user
01294_lazy_database_concurrent

View File

@ -255,7 +255,7 @@ windowFunnel(window, [mode, [mode, ... ]])(timestamp, cond1, cond2, ..., condN)
- `window` — Length of the sliding window, it is the time interval between the first and the last condition. The unit of `window` depends on the `timestamp` itself and varies. Determined using the expression `timestamp of cond1 <= timestamp of cond2 <= ... <= timestamp of condN <= timestamp of cond1 + window`.
- `mode` — It is an optional argument. One or more modes can be set.
- `'strict'` — If same condition holds for sequence of events then such non-unique events would be skipped.
- `'strict_deduplication'` — If the same condition holds for the sequence of events, then such repeating event interrupts further processing.
- `'strict_order'` — Don't allow interventions of other events. E.g. in the case of `A->B->D->C`, it stops finding `A->B->C` at the `D` and the max event level is 2.
- `'strict_increase'` — Apply conditions only to events with strictly increasing timestamps.

View File

@ -143,7 +143,9 @@ It works faster than intHash32. Average quality.
## SHA256 {#sha256}
Calculates SHA-1, SHA-224, or SHA-256 from a string and returns the resulting set of bytes as FixedString(20), FixedString(28), or FixedString(32).
## SHA512 {#sha512}
Calculates SHA-1, SHA-224, SHA-256 or SHA-512 from a string and returns the resulting set of bytes as FixedString(20), FixedString(28), FixedString(32), or FixedString(64).
The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million).
We recommend using this function only in cases when you need a specific hash function and you cant select it.
Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in SELECTS.

View File

@ -357,6 +357,7 @@ void Server::createServer(const std::string & listen_host, const char * port_nam
try
{
func(port);
global_context->registerServerPort(port_name, port);
}
catch (const Poco::Exception &)
{

View File

@ -137,8 +137,8 @@ class AggregateFunctionWindowFunnel final
private:
UInt64 window;
UInt8 events_size;
/// When the 'strict' is set, it applies conditions only for the not repeating values.
bool strict;
/// When the 'strict_deduplication' is set, it applies conditions only for the not repeating values.
bool strict_deduplication;
/// When the 'strict_order' is set, it doesn't allow interventions of other events.
/// In the case of 'A->B->D->C', it stops finding 'A->B->C' at the 'D' and the max event level is 2.
@ -150,7 +150,7 @@ private:
/// Loop through the entire events_list, update the event timestamp value
/// The level path must be 1---2---3---...---check_events_size, find the max event level that satisfied the path in the sliding window.
/// If found, returns the max event level, else return 0.
/// The Algorithm complexity is O(n).
/// The algorithm works in O(n) time, but the overall function works in O(n * log(n)) due to sorting.
UInt8 getEventLevel(Data & data) const
{
if (data.size() == 0)
@ -163,10 +163,10 @@ private:
/// events_timestamp stores the timestamp of the first and previous i-th level event happen within time window
std::vector<std::optional<std::pair<UInt64, UInt64>>> events_timestamp(events_size);
bool first_event = false;
for (const auto & pair : data.events_list)
for (size_t i = 0; i < data.events_list.size(); ++i)
{
const T & timestamp = pair.first;
const auto & event_idx = pair.second - 1;
const T & timestamp = data.events_list[i].first;
const auto & event_idx = data.events_list[i].second - 1;
if (strict_order && event_idx == -1)
{
if (first_event)
@ -179,9 +179,9 @@ private:
events_timestamp[0] = std::make_pair(timestamp, timestamp);
first_event = true;
}
else if (strict && events_timestamp[event_idx].has_value())
else if (strict_deduplication && events_timestamp[event_idx].has_value())
{
return event_idx + 1;
return data.events_list[i - 1].second;
}
else if (strict_order && first_event && !events_timestamp[event_idx - 1].has_value())
{
@ -226,18 +226,20 @@ public:
events_size = arguments.size() - 1;
window = params.at(0).safeGet<UInt64>();
strict = false;
strict_deduplication = false;
strict_order = false;
strict_increase = false;
for (size_t i = 1; i < params.size(); ++i)
{
String option = params.at(i).safeGet<String>();
if (option == "strict")
strict = true;
if (option == "strict_deduplication")
strict_deduplication = true;
else if (option == "strict_order")
strict_order = true;
else if (option == "strict_increase")
strict_increase = true;
else if (option == "strict")
throw Exception{"strict is replaced with strict_deduplication in Aggregate function " + getName(), ErrorCodes::BAD_ARGUMENTS};
else
throw Exception{"Aggregate function " + getName() + " doesn't support a parameter: " + option, ErrorCodes::BAD_ARGUMENTS};
}

View File

@ -80,8 +80,3 @@ target_link_libraries (average PRIVATE clickhouse_common_io)
add_executable (shell_command_inout shell_command_inout.cpp)
target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io)
if (ENABLE_FUZZING)
add_executable(YAML_fuzzer YAML_fuzzer.cpp ${SRCS})
target_link_libraries(YAML_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE})
endif ()

View File

@ -1,39 +0,0 @@
#include <iostream>
#include <fstream>
#include <string>
#include <cstdio>
#include <time.h>
#include <filesystem>
extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
{
/// How to test:
/// build ClickHouse with YAML_fuzzer.cpp
/// ./YAML_fuzzer YAML_CORPUS
/// where YAML_CORPUS is a directory with different YAML configs for libfuzzer
char file_name[L_tmpnam];
if (!std::tmpnam(file_name))
{
std::cerr << "Cannot create temp file!\n";
return 1;
}
std::string input = std::string(reinterpret_cast<const char*>(data), size);
DB::YAMLParser parser;
{
std::ofstream temp_file(file_name);
temp_file << input;
}
try
{
DB::YAMLParser::parse(std::string(file_name));
}
catch (...)
{
std::cerr << "YAML_fuzzer failed: " << getCurrentExceptionMessage() << std::endl;
return 1;
}
return 0;
}

View File

@ -1,3 +1,18 @@
if(ENABLE_EXAMPLES)
if (ENABLE_FUZZING)
include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
add_headers_and_sources(fuzz_compression .)
# Remove this file, because it has dependencies on DataTypes
list(REMOVE_ITEM ${fuzz_compression_sources} CompressionFactoryAdditions.cpp)
add_library(fuzz_compression ${fuzz_compression_headers} ${fuzz_compression_sources})
target_link_libraries(fuzz_compression PUBLIC clickhouse_parsers clickhouse_common_io common lz4)
endif()
if (ENABLE_EXAMPLES)
add_subdirectory(examples)
endif()
if (ENABLE_FUZZING)
add_subdirectory(fuzzers)
endif()

View File

@ -22,13 +22,10 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_CODEC;
extern const int BAD_ARGUMENTS;
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS;
}
static constexpr auto DEFAULT_CODEC_NAME = "Default";
CompressionCodecPtr CompressionCodecFactory::getDefaultCodec() const
{
return default_codec;
@ -49,184 +46,6 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std
}
}
void CompressionCodecFactory::validateCodec(
const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs) const
{
if (family_name.empty())
throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS);
if (level)
{
auto literal = std::make_shared<ASTLiteral>(static_cast<UInt64>(*level));
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)),
{}, sanity_check, allow_experimental_codecs);
}
else
{
auto identifier = std::make_shared<ASTIdentifier>(Poco::toUpper(family_name));
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier),
{}, sanity_check, allow_experimental_codecs);
}
}
ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const
{
if (const auto * func = ast->as<ASTFunction>())
{
ASTPtr codecs_descriptions = std::make_shared<ASTExpressionList>();
bool is_compression = false;
bool has_none = false;
std::optional<size_t> generic_compression_codec_pos;
std::set<size_t> post_processing_codecs;
bool can_substitute_codec_arguments = true;
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
{
const auto & inner_codec_ast = func->arguments->children[i];
String codec_family_name;
ASTPtr codec_arguments;
if (const auto * family_name = inner_codec_ast->as<ASTIdentifier>())
{
codec_family_name = family_name->name();
codec_arguments = {};
}
else if (const auto * ast_func = inner_codec_ast->as<ASTFunction>())
{
codec_family_name = ast_func->name;
codec_arguments = ast_func->arguments;
}
else
throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
/// Default codec replaced with current default codec which may depend on different
/// settings (and properties of data) in runtime.
CompressionCodecPtr result_codec;
if (codec_family_name == DEFAULT_CODEC_NAME)
{
if (codec_arguments != nullptr)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"{} codec cannot have any arguments, it's just an alias for codec specified in config.xml", DEFAULT_CODEC_NAME);
result_codec = default_codec;
codecs_descriptions->children.emplace_back(std::make_shared<ASTIdentifier>(DEFAULT_CODEC_NAME));
}
else
{
if (column_type)
{
CompressionCodecPtr prev_codec;
IDataType::StreamCallbackWithType callback = [&](
const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type)
{
if (ISerialization::isSpecialCompressionAllowed(substream_path))
{
result_codec = getImpl(codec_family_name, codec_arguments, &substream_type);
/// Case for column Tuple, which compressed with codec which depends on data type, like Delta.
/// We cannot substitute parameters for such codecs.
if (prev_codec && prev_codec->getHash() != result_codec->getHash())
can_substitute_codec_arguments = false;
prev_codec = result_codec;
}
};
ISerialization::SubstreamPath stream_path;
column_type->enumerateStreams(column_type->getDefaultSerialization(), callback, stream_path);
if (!result_codec)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName());
}
else
{
result_codec = getImpl(codec_family_name, codec_arguments, nullptr);
}
if (!allow_experimental_codecs && result_codec->isExperimental())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Codec {} is experimental and not meant to be used in production."
" You can enable it with the 'allow_experimental_codecs' setting.",
codec_family_name);
codecs_descriptions->children.emplace_back(result_codec->getCodecDesc());
}
is_compression |= result_codec->isCompression();
has_none |= result_codec->isNone();
if (!generic_compression_codec_pos && result_codec->isGenericCompression())
generic_compression_codec_pos = i;
if (result_codec->isPostProcessing())
post_processing_codecs.insert(i);
}
String codec_description = queryToString(codecs_descriptions);
if (sanity_check)
{
if (codecs_descriptions->children.size() > 1 && has_none)
throw Exception(
"It does not make sense to have codec NONE along with other compression codecs: " + codec_description
+ ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS);
/// Allow to explicitly specify single NONE codec if user don't want any compression.
/// But applying other transformations solely without compression (e.g. Delta) does not make sense.
/// It's okay to apply post-processing codecs solely without anything else.
if (!is_compression && !has_none && post_processing_codecs.size() != codecs_descriptions->children.size())
throw Exception(
"Compression codec " + codec_description
+ " does not compress anything."
" You may want to add generic compression algorithm after other transformations, like: "
+ codec_description
+ ", LZ4."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS);
/// It does not make sense to apply any non-post-processing codecs
/// after post-processing one.
if (!post_processing_codecs.empty() &&
*post_processing_codecs.begin() != codecs_descriptions->children.size() - post_processing_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any non-post-processing codecs after"
" post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'"
" to skip this check).", ErrorCodes::BAD_ARGUMENTS);
/// It does not make sense to apply any transformations after generic compression algorithm
/// So, generic compression can be only one and only at the end.
if (generic_compression_codec_pos &&
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any transformations after generic compression algorithm."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS);
}
/// For columns with nested types like Tuple(UInt32, UInt64) we
/// obviously cannot substitute parameters for codecs which depend on
/// data type, because for the first column Delta(4) is suitable and
/// Delta(8) for the second. So we should leave codec description as is
/// and deduce them in get method for each subtype separately. For all
/// other types it's better to substitute parameters, for better
/// readability and backward compatibility.
if (can_substitute_codec_arguments)
{
std::shared_ptr<ASTFunction> result = std::make_shared<ASTFunction>();
result->name = "CODEC";
result->arguments = codecs_descriptions;
return result;
}
else
{
return ast;
}
}
throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC);
}
CompressionCodecPtr CompressionCodecFactory::get(
const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default, bool only_generic) const

View File

@ -14,6 +14,8 @@
namespace DB
{
static constexpr auto DEFAULT_CODEC_NAME = "Default";
class ICompressionCodec;
using CompressionCodecPtr = std::shared_ptr<ICompressionCodec>;

View File

@ -0,0 +1,214 @@
/**
* This file contains a part of CompressionCodecFactory methods definitions and
* is needed only because they have dependencies on DataTypes.
* They are not useful for fuzzers, so we leave them in other translation unit.
*/
#include <Compression/CompressionFactory.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeNested.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int UNKNOWN_CODEC;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
void CompressionCodecFactory::validateCodec(
const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs) const
{
if (family_name.empty())
throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS);
if (level)
{
auto literal = std::make_shared<ASTLiteral>(static_cast<UInt64>(*level));
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)),
{}, sanity_check, allow_experimental_codecs);
}
else
{
auto identifier = std::make_shared<ASTIdentifier>(Poco::toUpper(family_name));
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier),
{}, sanity_check, allow_experimental_codecs);
}
}
ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const
{
if (const auto * func = ast->as<ASTFunction>())
{
ASTPtr codecs_descriptions = std::make_shared<ASTExpressionList>();
bool is_compression = false;
bool has_none = false;
std::optional<size_t> generic_compression_codec_pos;
std::set<size_t> post_processing_codecs;
bool can_substitute_codec_arguments = true;
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
{
const auto & inner_codec_ast = func->arguments->children[i];
String codec_family_name;
ASTPtr codec_arguments;
if (const auto * family_name = inner_codec_ast->as<ASTIdentifier>())
{
codec_family_name = family_name->name();
codec_arguments = {};
}
else if (const auto * ast_func = inner_codec_ast->as<ASTFunction>())
{
codec_family_name = ast_func->name;
codec_arguments = ast_func->arguments;
}
else
throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
/// Default codec replaced with current default codec which may depend on different
/// settings (and properties of data) in runtime.
CompressionCodecPtr result_codec;
if (codec_family_name == DEFAULT_CODEC_NAME)
{
if (codec_arguments != nullptr)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"{} codec cannot have any arguments, it's just an alias for codec specified in config.xml", DEFAULT_CODEC_NAME);
result_codec = default_codec;
codecs_descriptions->children.emplace_back(std::make_shared<ASTIdentifier>(DEFAULT_CODEC_NAME));
}
else
{
if (column_type)
{
CompressionCodecPtr prev_codec;
IDataType::StreamCallbackWithType callback = [&](
const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type)
{
if (ISerialization::isSpecialCompressionAllowed(substream_path))
{
result_codec = getImpl(codec_family_name, codec_arguments, &substream_type);
/// Case for column Tuple, which compressed with codec which depends on data type, like Delta.
/// We cannot substitute parameters for such codecs.
if (prev_codec && prev_codec->getHash() != result_codec->getHash())
can_substitute_codec_arguments = false;
prev_codec = result_codec;
}
};
ISerialization::SubstreamPath stream_path;
column_type->enumerateStreams(column_type->getDefaultSerialization(), callback, stream_path);
if (!result_codec)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName());
}
else
{
result_codec = getImpl(codec_family_name, codec_arguments, nullptr);
}
if (!allow_experimental_codecs && result_codec->isExperimental())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Codec {} is experimental and not meant to be used in production."
" You can enable it with the 'allow_experimental_codecs' setting.",
codec_family_name);
codecs_descriptions->children.emplace_back(result_codec->getCodecDesc());
}
is_compression |= result_codec->isCompression();
has_none |= result_codec->isNone();
if (!generic_compression_codec_pos && result_codec->isGenericCompression())
generic_compression_codec_pos = i;
if (result_codec->isPostProcessing())
post_processing_codecs.insert(i);
}
String codec_description = queryToString(codecs_descriptions);
if (sanity_check)
{
if (codecs_descriptions->children.size() > 1 && has_none)
throw Exception(
"It does not make sense to have codec NONE along with other compression codecs: " + codec_description
+ ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS);
/// Allow to explicitly specify single NONE codec if user don't want any compression.
/// But applying other transformations solely without compression (e.g. Delta) does not make sense.
/// It's okay to apply post-processing codecs solely without anything else.
if (!is_compression && !has_none && post_processing_codecs.size() != codecs_descriptions->children.size())
throw Exception(
"Compression codec " + codec_description
+ " does not compress anything."
" You may want to add generic compression algorithm after other transformations, like: "
+ codec_description
+ ", LZ4."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS);
/// It does not make sense to apply any non-post-processing codecs
/// after post-processing one.
if (!post_processing_codecs.empty() &&
*post_processing_codecs.begin() != codecs_descriptions->children.size() - post_processing_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any non-post-processing codecs after"
" post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'"
" to skip this check).", ErrorCodes::BAD_ARGUMENTS);
/// It does not make sense to apply any transformations after generic compression algorithm
/// So, generic compression can be only one and only at the end.
if (generic_compression_codec_pos &&
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any transformations after generic compression algorithm."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS);
}
/// For columns with nested types like Tuple(UInt32, UInt64) we
/// obviously cannot substitute parameters for codecs which depend on
/// data type, because for the first column Delta(4) is suitable and
/// Delta(8) for the second. So we should leave codec description as is
/// and deduce them in get method for each subtype separately. For all
/// other types it's better to substitute parameters, for better
/// readability and backward compatibility.
if (can_substitute_codec_arguments)
{
std::shared_ptr<ASTFunction> result = std::make_shared<ASTFunction>();
result->name = "CODEC";
result->arguments = codecs_descriptions;
return result;
}
else
{
return ast;
}
}
throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC);
}
}

View File

@ -3,8 +3,3 @@ target_link_libraries (compressed_buffer PRIVATE dbms)
add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp)
target_link_libraries (cached_compressed_read_buffer PRIVATE dbms)
if (ENABLE_FUZZING)
add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp)
target_link_libraries (compressed_buffer_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE})
endif ()

View File

@ -0,0 +1,2 @@
add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp)
target_link_libraries (compressed_buffer_fuzzer PRIVATE fuzz_compression clickhouse_common_io ${LIB_FUZZING_ENGINE})

View File

@ -1,3 +1,7 @@
if (ENABLE_EXAMPLES)
add_subdirectory(examples)
endif ()
if (ENABLE_FUZZING)
add_subdirectory(fuzzers)
endif()

View File

@ -55,7 +55,7 @@ class IColumn;
M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \
M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \
M(Seconds, drain_timeout, DBMS_DEFAULT_DRAIN_TIMEOUT_SEC, "", 0) \
M(Seconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \
M(Seconds, tcp_keep_alive_timeout, 290 /* less than DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC */, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \
M(Milliseconds, hedged_connection_timeout_ms, DBMS_DEFAULT_HEDGED_CONNECTION_TIMEOUT_MS, "Connection timeout for establishing connection with replica for Hedged requests", 0) \
M(Milliseconds, receive_data_timeout_ms, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_MS, "Connection timeout for receiving first packet of data or packet with positive progress from replica", 0) \
M(Bool, use_hedged_requests, true, "Use hedged requests for distributed queries", 0) \

View File

@ -8,11 +8,6 @@ target_link_libraries (field PRIVATE dbms)
add_executable (string_ref_hash string_ref_hash.cpp)
target_link_libraries (string_ref_hash PRIVATE clickhouse_common_io)
if (ENABLE_FUZZING)
add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp)
target_link_libraries (names_and_types_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE})
endif ()
add_executable (mysql_protocol mysql_protocol.cpp)
target_link_libraries (mysql_protocol PRIVATE dbms)
if(USE_SSL)

View File

@ -0,0 +1,2 @@
add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp)
target_link_libraries (names_and_types_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE})

View File

@ -26,23 +26,6 @@ namespace ErrorCodes
IDataType::~IDataType() = default;
String IDataType::getName() const
{
if (custom_name)
{
return custom_name->getName();
}
else
{
return doGetName();
}
}
String IDataType::doGetName() const
{
return getFamilyName();
}
void IDataType::updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint)
{
/// Update the average value size hint if amount of read rows isn't too small

View File

@ -62,7 +62,13 @@ public:
/// static constexpr bool is_parametric = false;
/// Name of data type (examples: UInt64, Array(String)).
String getName() const;
String getName() const
{
if (custom_name)
return custom_name->getName();
else
return doGetName();
}
/// Name of data type family (example: FixedString, Array).
virtual const char * getFamilyName() const = 0;
@ -105,7 +111,7 @@ public:
void enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback) const { enumerateStreams(serialization, callback, {}); }
protected:
virtual String doGetName() const;
virtual String doGetName() const { return getFamilyName(); }
virtual SerializationPtr doGetDefaultSerialization() const = 0;
DataTypePtr getTypeForSubstream(const ISerialization::SubstreamPath & substream_path) const;

View File

@ -14,6 +14,7 @@ void registerFunctionsHashing(FunctionFactory & factory)
factory.registerFunction<FunctionSHA1>();
factory.registerFunction<FunctionSHA224>();
factory.registerFunction<FunctionSHA256>();
factory.registerFunction<FunctionSHA512>();
#endif
factory.registerFunction<FunctionSipHash64>();
factory.registerFunction<FunctionSipHash128>();

View File

@ -193,6 +193,20 @@ struct SHA256Impl
SHA256_Final(out_char_data, &ctx);
}
};
struct SHA512Impl
{
static constexpr auto name = "SHA512";
enum { length = 64 };
static void apply(const char * begin, const size_t size, unsigned char * out_char_data)
{
SHA512_CTX ctx;
SHA512_Init(&ctx);
SHA512_Update(&ctx, reinterpret_cast<const unsigned char *>(begin), size);
SHA512_Final(out_char_data, &ctx);
}
};
#endif
struct SipHash64Impl
@ -1318,6 +1332,7 @@ using FunctionMD5 = FunctionStringHashFixedString<MD5Impl>;
using FunctionSHA1 = FunctionStringHashFixedString<SHA1Impl>;
using FunctionSHA224 = FunctionStringHashFixedString<SHA224Impl>;
using FunctionSHA256 = FunctionStringHashFixedString<SHA256Impl>;
using FunctionSHA512 = FunctionStringHashFixedString<SHA512Impl>;
#endif
using FunctionSipHash128 = FunctionStringHashFixedString<SipHash128Impl>;
using FunctionCityHash64 = FunctionAnyHash<ImplCityHash64>;

View File

@ -696,6 +696,8 @@ struct JSONExtractTree
{
if (element.isString())
return JSONExtractStringImpl<JSONParser>::insertResultToColumn(dest, element, {});
else if (element.isNull())
return false;
else
return JSONExtractRawImpl<JSONParser>::insertResultToColumn(dest, element, {});
}

View File

@ -0,0 +1,136 @@
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Interpreters/Context.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
class ExecutableFunctionGetServerPort : public IExecutableFunction
{
public:
explicit ExecutableFunctionGetServerPort(UInt16 port_) : port(port_) {}
String getName() const override { return "getServerPort"; }
bool useDefaultImplementationForNulls() const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
{
return DataTypeNumber<UInt16>().createColumnConst(input_rows_count, port);
}
private:
UInt16 port;
};
class FunctionBaseGetServerPort : public IFunctionBase
{
public:
explicit FunctionBaseGetServerPort(bool is_distributed_, UInt16 port_, DataTypes argument_types_, DataTypePtr return_type_)
: is_distributed(is_distributed_), port(port_), argument_types(std::move(argument_types_)), return_type(std::move(return_type_))
{
}
String getName() const override { return "getServerPort"; }
const DataTypes & getArgumentTypes() const override
{
return argument_types;
}
const DataTypePtr & getResultType() const override
{
return return_type;
}
bool isDeterministic() const override { return false; }
bool isDeterministicInScopeOfQuery() const override { return true; }
bool isSuitableForConstantFolding() const override { return !is_distributed; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override
{
return std::make_unique<ExecutableFunctionGetServerPort>(port);
}
private:
bool is_distributed;
UInt16 port;
DataTypes argument_types;
DataTypePtr return_type;
};
class GetServerPortOverloadResolver : public IFunctionOverloadResolver, WithContext
{
public:
static constexpr auto name = "getServerPort";
String getName() const override { return name; }
static FunctionOverloadResolverPtr create(ContextPtr context_)
{
return std::make_unique<GetServerPortOverloadResolver>(context_);
}
explicit GetServerPortOverloadResolver(ContextPtr context_) : WithContext(context_) {}
size_t getNumberOfArguments() const override { return 1; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; }
bool isDeterministic() const override { return false; }
bool isDeterministicInScopeOfQuery() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override
{
size_t number_of_arguments = data_types.size();
if (number_of_arguments != 1)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be 1",
getName(),
number_of_arguments);
return std::make_shared<DataTypeNumber<UInt16>>();
}
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
{
if (!isString(arguments[0].type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The argument of function {} should be a constant string with the name of a setting",
getName());
const auto * column = arguments[0].column.get();
if (!column || !checkAndGetColumnConstStringOrFixedString(column))
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"The argument of function {} should be a constant string with the name of a setting",
getName());
String port_name{column->getDataAt(0)};
auto port = getContext()->getServerPort(port_name);
DataTypes argument_types;
argument_types.emplace_back(arguments.back().type);
return std::make_unique<FunctionBaseGetServerPort>(getContext()->isDistributed(), port, argument_types, return_type);
}
};
}
void registerFunctionGetServerPort(FunctionFactory & factory)
{
factory.registerFunction<GetServerPortOverloadResolver>();
}
}

View File

@ -71,6 +71,7 @@ void registerFunctionHasThreadFuzzer(FunctionFactory &);
void registerFunctionInitializeAggregation(FunctionFactory &);
void registerFunctionErrorCodeToName(FunctionFactory &);
void registerFunctionTcpPort(FunctionFactory &);
void registerFunctionGetServerPort(FunctionFactory &);
void registerFunctionByteSize(FunctionFactory &);
void registerFunctionFile(FunctionFactory & factory);
void registerFunctionConnectionId(FunctionFactory & factory);
@ -150,6 +151,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
registerFunctionInitializeAggregation(factory);
registerFunctionErrorCodeToName(factory);
registerFunctionTcpPort(factory);
registerFunctionGetServerPort(factory);
registerFunctionByteSize(factory);
registerFunctionFile(factory);
registerFunctionConnectionId(factory);

View File

@ -228,6 +228,8 @@ struct ContextSharedPart
ConfigurationPtr clusters_config; /// Stores updated configs
mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config
std::map<String, UInt16> server_ports;
bool shutdown_called = false;
Stopwatch uptime_watch;
@ -1798,6 +1800,20 @@ std::optional<UInt16> Context::getTCPPortSecure() const
return {};
}
void Context::registerServerPort(String port_name, UInt16 port)
{
shared->server_ports.emplace(std::move(port_name), port);
}
UInt16 Context::getServerPort(const String & port_name) const
{
auto it = shared->server_ports.find(port_name);
if (it == shared->server_ports.end())
throw Exception(ErrorCodes::BAD_GET, "There is no port named {}", port_name);
else
return it->second;
}
std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) const
{
auto res = getClusters()->getCluster(cluster_name);

View File

@ -580,6 +580,11 @@ public:
std::optional<UInt16> getTCPPortSecure() const;
/// Register server ports during server starting up. No lock is held.
void registerServerPort(String port_name, UInt16 port);
UInt16 getServerPort(const String & port_name) const;
/// For methods below you may need to acquire the context lock by yourself.
ContextMutablePtr getQueryContext() const;

View File

@ -12,3 +12,7 @@ endif ()
if(ENABLE_EXAMPLES)
add_subdirectory(examples)
endif()
if (ENABLE_FUZZING)
add_subdirectory(fuzzers)
endif()

View File

@ -8,14 +8,3 @@ target_link_libraries(select_parser PRIVATE clickhouse_parsers)
add_executable(create_parser create_parser.cpp ${SRCS})
target_link_libraries(create_parser PRIVATE clickhouse_parsers)
if (ENABLE_FUZZING)
add_executable(lexer_fuzzer lexer_fuzzer.cpp ${SRCS})
target_link_libraries(lexer_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE})
add_executable(select_parser_fuzzer select_parser_fuzzer.cpp ${SRCS})
target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE})
add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS})
target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE})
endif ()

View File

@ -0,0 +1,8 @@
add_executable(lexer_fuzzer lexer_fuzzer.cpp ${SRCS})
target_link_libraries(lexer_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE})
add_executable(select_parser_fuzzer select_parser_fuzzer.cpp ${SRCS})
target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE})
add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS})
target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE})

View File

@ -15,7 +15,10 @@ try
DB::ParserCreateQuery parser;
DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
DB::formatAST(*ast, std::cerr);
DB::WriteBufferFromOwnString wb;
DB::formatAST(*ast, wb);
std::cerr << wb.str() << std::endl;
return 0;
}

View File

@ -14,7 +14,10 @@ try
DB::ParserQueryWithOutput parser(input.data() + input.size());
DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
DB::formatAST(*ast, std::cerr);
DB::WriteBufferFromOwnString wb;
DB::formatAST(*ast, wb);
std::cerr << wb.str() << std::endl;
return 0;
}

View File

@ -1,6 +1,10 @@
add_subdirectory(MergeTree)
add_subdirectory(System)
if(ENABLE_EXAMPLES)
if (ENABLE_EXAMPLES)
add_subdirectory(examples)
endif()
if (ENABLE_FUZZING)
add_subdirectory(fuzzers)
endif()

View File

@ -23,10 +23,3 @@ target_link_libraries (transform_part_zk_nodes
string_utils
)
if (ENABLE_FUZZING)
add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.cpp)
target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE})
add_executable (columns_description_fuzzer columns_description_fuzzer.cpp)
target_link_libraries (columns_description_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE})
endif ()

View File

@ -0,0 +1,11 @@
add_executable (mergetree_checksum_fuzzer
mergetree_checksum_fuzzer.cpp
"${ClickHouse_SOURCE_DIR}/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp"
"${ClickHouse_SOURCE_DIR}/src/Compression/CompressedReadBuffer.cpp"
"${ClickHouse_SOURCE_DIR}/src/Compression/CompressedWriteBuffer.cpp"
)
target_link_libraries (mergetree_checksum_fuzzer PRIVATE clickhouse_common_io fuzz_compression ${LIB_FUZZING_ENGINE})
add_executable (columns_description_fuzzer columns_description_fuzzer.cpp)
target_link_libraries (columns_description_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE})

View File

@ -1841,6 +1841,10 @@ class ClickHouseInstance:
build_opts = self.query("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'")
return "-fsanitize={}".format(sanitizer_name) in build_opts
def is_debug_build(self):
build_opts = self.query("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'")
return 'NDEBUG' not in build_opts
def is_built_with_thread_sanitizer(self):
return self.is_built_with_sanitizer('thread')
@ -2029,6 +2033,37 @@ class ClickHouseInstance:
return None
return None
def restart_with_original_version(self, stop_start_wait_sec=300, callback_onstop=None, signal=15):
if not self.stay_alive:
raise Exception("Cannot restart not stay alive container")
self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(signal)], user='root')
retries = int(stop_start_wait_sec / 0.5)
local_counter = 0
# wait stop
while local_counter < retries:
if not self.get_process_pid("clickhouse server"):
break
time.sleep(0.5)
local_counter += 1
# force kill if server hangs
if self.get_process_pid("clickhouse server"):
# server can die before kill, so don't throw exception, it's expected
self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(9)], nothrow=True, user='root')
if callback_onstop:
callback_onstop(self)
self.exec_in_container(
["bash", "-c", "cp /usr/share/clickhouse_original /usr/bin/clickhouse && chmod 777 /usr/bin/clickhouse"],
user='root')
self.exec_in_container(["bash", "-c",
"cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse"],
user='root')
self.exec_in_container(["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], user=str(os.getuid()))
# wait start
assert_eq_with_retry(self, "select 1", "1", retry_count=retries)
def restart_with_latest_version(self, stop_start_wait_sec=300, callback_onstop=None, signal=15):
if not self.stay_alive:
raise Exception("Cannot restart not stay alive container")
@ -2049,6 +2084,9 @@ class ClickHouseInstance:
if callback_onstop:
callback_onstop(self)
self.exec_in_container(
["bash", "-c", "cp /usr/bin/clickhouse /usr/share/clickhouse_original"],
user='root')
self.exec_in_container(
["bash", "-c", "cp /usr/share/clickhouse_fresh /usr/bin/clickhouse && chmod 777 /usr/bin/clickhouse"],
user='root')

View File

@ -53,3 +53,9 @@ def test_backward_compatability(start_cluster):
node1.restart_with_latest_version()
assert (node1.query("SELECT avgMerge(x) FROM state") == '2.5\n')
node1.query("drop table tab")
node1.query("drop table state")
node2.query("drop table tab")
node3.query("drop table tab")
node4.query("drop table tab")

View File

@ -5,7 +5,7 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
cluster = ClickHouseCluster(__file__, name="skipping_indices")
node = cluster.add_instance('node', image='yandex/clickhouse-server', tag='21.6', stay_alive=True, with_installed_binary=True)
@ -41,4 +41,4 @@ def test_index(start_cluster):
node.query("""
SELECT * FROM data WHERE value = 20000 SETTINGS force_data_skipping_indices = 'value_index' SETTINGS force_data_skipping_indices = 'value_index', max_rows_to_read=1;
DROP TABLE data;
""")
""")

View File

@ -30,3 +30,7 @@ def test_detach_part_wrong_partition_id(start_cluster):
num_detached = node_21_6.query("select count() from system.detached_parts")
assert num_detached == '1\n'
node_21_6.restart_with_original_version()
node_21_6.query("drop table tab SYNC")

View File

@ -27,3 +27,6 @@ def test_select_aggregate_alias_column(start_cluster):
node1.query("select sum(x_alias) from remote('node{1,2}', default, tab)")
node2.query("select sum(x_alias) from remote('node{1,2}', default, tab)")
node1.query("drop table tab")
node2.query("drop table tab")

View File

@ -29,3 +29,5 @@ def test_backward_compatability(start_cluster):
"select s, count() from remote('node{1,2}', default, tab) group by s order by toUInt64(s) limit 50")
print(res)
assert res == ''.join('{}\t2\n'.format(i) for i in range(50))
node1.query("drop table tab")
node2.query("drop table tab")

View File

@ -3,6 +3,7 @@ import os.path
import timeit
import pytest
import logging
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
from helpers.test_tools import TSV
@ -11,6 +12,8 @@ cluster = ClickHouseCluster(__file__)
NODES = {'node' + str(i): None for i in (1, 2)}
IS_DEBUG = False
CREATE_TABLES_SQL = '''
CREATE DATABASE test;
@ -104,6 +107,11 @@ def started_cluster(request):
try:
cluster.start()
if cluster.instances["node1"].is_debug_build():
global IS_DEBUG
IS_DEBUG = True
logging.warning("Debug build is too slow to show difference in timings. We disable checks.")
for node_id, node in list(NODES.items()):
node.query(CREATE_TABLES_SQL)
node.query(INSERT_SQL_TEMPLATE.format(node_id=node_id))
@ -133,8 +141,9 @@ def _check_timeout_and_exception(node, user, query_base, query):
# And it should timeout no faster than:
measured_timeout = timeit.default_timer() - start
assert expected_timeout - measured_timeout <= TIMEOUT_MEASUREMENT_EPS
assert measured_timeout - expected_timeout <= TIMEOUT_DIFF_UPPER_BOUND[user][query_base]
if not IS_DEBUG:
assert expected_timeout - measured_timeout <= TIMEOUT_MEASUREMENT_EPS
assert measured_timeout - expected_timeout <= TIMEOUT_DIFF_UPPER_BOUND[user][query_base]
# And exception should reflect connection attempts:
_check_exception(exception, repeats)

View File

@ -37,6 +37,9 @@ def cluster():
with_hdfs=True)
logging.info("Starting cluster...")
cluster.start()
if cluster.instances["node1"].is_debug_build():
# https://github.com/ClickHouse/ClickHouse/issues/27814
pytest.skip("libhdfs3 calls rand function which does not pass harmful check in debug build")
logging.info("Cluster started")
fs = HdfsClient(hosts=cluster.hdfs_ip)

View File

@ -180,28 +180,6 @@ def avro_confluent_message(schema_registry_client, value):
})
return serializer.encode_record_with_schema('test_subject', schema, value)
# Fixtures
@pytest.fixture(scope="module")
def kafka_cluster():
try:
global kafka_id
cluster.start()
kafka_id = instance.cluster.kafka_docker_id
print(("kafka_id is {}".format(kafka_id)))
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def kafka_setup_teardown():
instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;')
wait_kafka_is_available() # ensure kafka is alive
kafka_producer_send_heartbeat_msg() # ensure python kafka client is ok
# print("kafka is available - running test")
yield # run test
# Tests
def test_kafka_settings_old_syntax(kafka_cluster):
@ -699,6 +677,8 @@ def describe_consumer_group(kafka_cluster, name):
def kafka_cluster():
try:
cluster.start()
kafka_id = instance.cluster.kafka_docker_id
print(("kafka_id is {}".format(kafka_id)))
yield cluster
finally:
cluster.shutdown()
@ -1129,6 +1109,7 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster):
def test_kafka_materialized_view(kafka_cluster):
instance.query('''
DROP TABLE IF EXISTS test.view;
DROP TABLE IF EXISTS test.consumer;

View File

@ -55,6 +55,9 @@ def kafka_produce(kafka_cluster, topic, messages, timestamp=None):
def kafka_cluster():
try:
cluster.start()
if instance.is_debug_build():
# https://github.com/ClickHouse/ClickHouse/issues/27651
pytest.skip("librdkafka calls system function for kinit which does not pass harmful check in debug build")
yield cluster
finally:
cluster.shutdown()

View File

@ -37,6 +37,7 @@
[5, 2]
[6, 1]
[7, 1]
[1]
[1, 2]
[2, 2]
[3, 0]

View File

@ -43,7 +43,7 @@ drop table if exists funnel_test_strict;
create table funnel_test_strict (timestamp UInt32, event UInt32) engine=Memory;
insert into funnel_test_strict values (00,1000),(10,1001),(20,1002),(30,1003),(40,1004),(50,1005),(51,1005),(60,1006),(70,1007),(80,1008);
select 6 = windowFunnel(10000, 'strict')(timestamp, event = 1000, event = 1001, event = 1002, event = 1003, event = 1004, event = 1005, event = 1006) from funnel_test_strict;
select 6 = windowFunnel(10000, 'strict_deduplication')(timestamp, event = 1000, event = 1001, event = 1002, event = 1003, event = 1004, event = 1005, event = 1006) from funnel_test_strict;
select 7 = windowFunnel(10000)(timestamp, event = 1000, event = 1001, event = 1002, event = 1003, event = 1004, event = 1005, event = 1006) from funnel_test_strict;
@ -62,11 +62,18 @@ insert into funnel_test_strict_order values (1, 5, 'a') (2, 5, 'a') (3, 5, 'b')
insert into funnel_test_strict_order values (1, 6, 'c') (2, 6, 'c') (3, 6, 'b') (4, 6, 'b') (5, 6, 'a') (6, 6, 'a');
select user, windowFunnel(86400)(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
select user, windowFunnel(86400, 'strict_order')(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
select user, windowFunnel(86400, 'strict', 'strict_order')(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
select user, windowFunnel(86400, 'strict_deduplication', 'strict_order')(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
insert into funnel_test_strict_order values (1, 7, 'a') (2, 7, 'c') (3, 7, 'b');
select user, windowFunnel(10, 'strict_order')(dt, event = 'a', event = 'b', event = 'c') as s from funnel_test_strict_order where user = 7 group by user format JSONCompactEachRow;
drop table funnel_test_strict_order;
--https://github.com/ClickHouse/ClickHouse/issues/27469
drop table if exists strict_BiteTheDDDD;
create table strict_BiteTheDDDD (ts UInt64, event String) engine = Log();
insert into strict_BiteTheDDDD values (1,'a') (2,'b') (3,'c') (4,'b') (5,'d');
select 3 = windowFunnel(86400, 'strict_deduplication')(ts, event='a', event='b', event='c', event='d') from strict_BiteTheDDDD format JSONCompactEachRow;
drop table strict_BiteTheDDDD;
drop table if exists funnel_test_non_null;
create table funnel_test_non_null (`dt` DateTime, `u` int, `a` Nullable(String), `b` Nullable(String)) engine = MergeTree() partition by dt order by u;
insert into funnel_test_non_null values (1, 1, 'a1', 'b1') (2, 1, 'a2', 'b2');

View File

@ -1 +1,2 @@
('123','456','[7,8,9]')
\N

View File

@ -1 +1,3 @@
select JSONExtract('{"a": "123", "b": 456, "c": [7, 8, 9]}', 'Tuple(a String, b String, c String)');
with '{"string_value":null}' as json select JSONExtract(json, 'string_value', 'Nullable(String)');

View File

@ -0,0 +1 @@
9000

View File

@ -0,0 +1,3 @@
select getServerPort('tcp_port');
select getServerPort('unknown'); -- { serverError 170 }

View File

@ -0,0 +1,22 @@
CF83E1357EEFB8BDF1542850D66D8007D620E4050B5715DC83F4A921D36CE9CE47D0D13C5D85F2B0FF8318D2877EEC2F63B931BD47417A81A538327AF927DA3E
DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F
5809F3ECB4AA006F71AF562D4381F2BF64EA0931FD530E939740D0C38F6EEB2A71FA0113A21C170569D8319B8C4DE8A1C1A5ABA1A1C5B23A886B06712D373B9E
6FDB5E5BCCBD093ECC48DD262A99E6B867D6F48E1DAE014D26428365E7529B0022F000CBF852BEA38F43A2034E8FE7555AC41B9EA9E27FE72F4E968926998EA8
8018978D8AAE19322205E4CACFA045CDF7A0C4A5773A93FD24331064AFC5726F324B76802AA6FC30DFC412A6E5C3EEF4693AE4E2D0A1EA24A2D3EC46439B7923
22F1080BFD1CEA5B86CFDEFE129D2FF67E756AA3DA0EADB4C6EBA86A0710F3C2E0F91815A28062BDA4FBF23E5FA1BD66A3CFDA37AC7354516943AB6E08E88106
08788919600C5C9B93704570858D8B64D5B335FAD2B23E28A5E68A08BCC4AD9BCA1991178FEA22BDAAA3C9C2FEB061FF4919C1C840DFA8188858D80EDCE98499
0EFD835B217A94C32F45B9C3720644D7A4A49DD3A74EDE4257CFBB74164C80382322B8EA7B6131747609BA1B2776E84AEC2820D9414FC5F66B6A42D3F90F1D9F
C161DA64EE30E86768040C4BB7B88C3FAE5A1E79CA2441B2DB66552B126C00F952D2D854DF2F5D56FB85ED57C5A39E95BFE485A6AF70963A81BE67DFDB96CFBF
F65B5379D687EA1FB38E32F8251F0FB270061DC6DF9AECA1E258A6F1BAFF70D4E979FC9315ED1FAE24D000CC0EF7937D02703B2061694B3DCA2308C5C0779184
DF25395F5FFF00E5B64BF5EEAFD94A1A21B4F4FC4FAE01DF48C27C77E0C9DC6FC253C7E7F32185146048190B73E3ED53BC76F626D24E9DE3FBA3EBC48B671CC0
F1DD6EFB26D0169259FBD53150D6E0C4F2E1FFF2F9DA6B77EC9AD065AD67D33F95411CC5E5B31CEAB970974E5C8E3C6376445D8422F052CABB121BCE1449C494
561A1DEDBB03FD632DBFDCC73E83A8DBD17D6E30B14642BAC8EBF3DF6622150A5E51ACC059E2C44EF71A00FBEEB062F2EE7A610E5462E6B4805449574AAE857F
D59943FE43CB50F2C3AE74F9BD09F72140519EA30C1A24E6F7A4BCBBD12860D954F9752A24FBAA1D606F362564D905E8B29806903B1542D948C2B8293FC09A59
E18A99CED7BD69979406C5A666336D7316E11240FF5F8C1632F17BEB1BD58838E0222CCD48D4227F0751592F1C731B4BB11C325F09FB15923F0334C64303D73A
0508B92D15B0750343F9B3E2218C7D54BB6D9A3ABA5FA941977832AA89300516505E91034E4ECD4812A7A58438251651C00974DDCD3C471B9ED02451871F4A48
ADFDC00D41881138C96581D43298C5724C20500C5A966144B51531FFB2FE6CE265BE3101CD19B985A2253A7B48EE4EC5EBD4B2876CF0E66F1095EB0A2521C525
758ABA28159AABFFF659E6698397AD7A9EBBA77AA3FFC25551B7CF2057930AD051D357866259E5288B0B837E3F5C4ED1D1229320595F914381903CD48DF76CB2
E7A061D9B066E2CA44CF959A76FC04D8B02998CB9D46A60C19E015EA9389F3F9595CBBC4CC46E1319B02B3269FBD21F62D15A6F13428F8389CB4507AF6DB6D89
3438D75650E1EDB8A11EF4F63A5DFF239A70B28B6A14F13FCFDD14D02BE8BD00E84DF956C159CFDC85D6E44DB62D00E9206F40453FFD9CC97C38449527D33FF6
DB229C3A53B0340E94EFDA4D03B54F161313699757CAC312F377B731AE6C62010E0C0010E78F73E6D6B0BB438F644D176244B2614897799F9FA3F85DA980C218
FDD9FD54050D95855B8E3A34F3A54E309E1CA87CD44A8506EB10051D1CA650DB64ABD0BE4F4F44E45F630C22CA270FA7694AC2261DF2EFD766B8CED53F285A27

View File

@ -0,0 +1,14 @@
SELECT hex(SHA512(''));
SELECT hex(SHA512('abc'));
DROP TABLE IF EXISTS defaults;
CREATE TABLE defaults
(
s FixedString(20)
)ENGINE = Memory();
INSERT INTO defaults SELECT s FROM generateRandom('s FixedString(20)', 1, 1, 1) LIMIT 20;
SELECT hex(SHA512(s)) FROM defaults;
DROP TABLE defaults;