Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into remote-connection

This commit is contained in:
kssenii 2021-10-06 11:54:28 +00:00
commit c96b403809
76 changed files with 3759 additions and 435 deletions

View File

@ -135,6 +135,7 @@ if (ENABLE_FUZZING)
# And we don't want to enable coverage for it.
message (STATUS "Fuzzing instrumentation enabled")
set (FUZZER "libfuzzer")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -nostdlib++")
endif()
# Global libraries

View File

@ -1,6 +1,7 @@
#pragma once
#include <map>
#include <tuple>
#include <mutex>
#include "FnTraits.h"
@ -17,7 +18,8 @@ struct CachedFn
{
private:
using Traits = FnTraits<decltype(Func)>;
using Key = typename Traits::DecayedArgs;
using DecayedArgs = TLMap<std::decay_t, typename Traits::Args>;
using Key = TLChangeRoot<std::tuple, DecayedArgs>;
using Result = typename Traits::Ret;
std::map<Key, Result> cache; // Can't use hashmap as tuples are unhashable by default

View File

@ -1,8 +1,6 @@
#pragma once
#include <tuple>
#include <concepts>
#include <type_traits>
#include "Typelist.h"
namespace detail
{
@ -16,8 +14,7 @@ struct FnTraits<R(A...)>
static constexpr bool value = std::is_invocable_r_v<R, F, A...>;
using Ret = R;
using Args = std::tuple<A...>;
using DecayedArgs = std::tuple<typename std::decay<A>::type...>;
using Args = Typelist<A...>;
};
template <class R, class ...A>

51
base/base/TypeName.h Normal file
View File

@ -0,0 +1,51 @@
#pragma once
#include "Decimal.h"
#include "UUID.h"
namespace DB
{
struct Array;
struct Tuple;
struct Map;
struct AggregateFunctionStateData;
/**
* Obtain type string representation from real type if possible.
* @example TypeName<UInt8> == "UInt8"
*/
template <class T> constexpr inline std::string_view TypeName;
#define TN_MAP(_A) \
template <> constexpr inline std::string_view TypeName<_A> = #_A;
TN_MAP(UInt8)
TN_MAP(UInt16)
TN_MAP(UInt32)
TN_MAP(UInt64)
TN_MAP(UInt128)
TN_MAP(UInt256)
TN_MAP(Int8)
TN_MAP(Int16)
TN_MAP(Int32)
TN_MAP(Int64)
TN_MAP(Int128)
TN_MAP(Int256)
TN_MAP(Float32)
TN_MAP(Float64)
TN_MAP(String)
TN_MAP(UUID)
TN_MAP(Decimal32)
TN_MAP(Decimal64)
TN_MAP(Decimal128)
TN_MAP(Decimal256)
TN_MAP(DateTime64)
TN_MAP(Array)
TN_MAP(Tuple)
TN_MAP(Map)
/// Special case
template <> constexpr inline std::string_view TypeName<AggregateFunctionStateData> = "AggregateFunctionState";
#undef TN_MAP
}

4
base/base/TypePair.h Normal file
View File

@ -0,0 +1,4 @@
#pragma once
template <class T, class V> struct TypePair { };
template <class T> struct Id { };

44
base/base/Typelist.h Normal file
View File

@ -0,0 +1,44 @@
#pragma once
#include <cstddef>
#include <type_traits>
#include <utility>
#include "defines.h"
#include "TypePair.h"
/// General-purpose typelist. Easy on compilation times as it does not use recursion.
template <class ...Args>
struct Typelist { static constexpr size_t size = sizeof...(Args); };
namespace TLUtils /// In some contexts it's more handy to use functions instead of aliases
{
template <class ...LArgs, class ...RArgs>
constexpr Typelist<LArgs..., RArgs...> concat(Typelist<LArgs...>, Typelist<RArgs...>) { return {}; }
template <class T, class ...Args>
constexpr Typelist<T, Args...> prepend(Typelist<Args...>) { return {}; }
template <class T, class ...Args>
constexpr Typelist<Args..., T> append(Typelist<Args...>) { return {}; }
template <template<class> class F, class ...Args>
constexpr Typelist<F<Args>...> map(Typelist<Args...>) { return {}; }
template <template<class...> class Root, class ...Args>
constexpr Root<Args...> changeRoot(Typelist<Args...>) { return {}; }
template <class F, class ...Args>
constexpr void forEach(Typelist<Args...>, F && f) { (std::forward<F>(f)(Id<Args>{}), ...); }
}
template <class TLLeft, class TLRight>
using TLConcat = decltype(TLUtils::concat(TLLeft{}, TLRight{}));
template <class T, class Typelist> using TLPrepend = decltype(TLUtils::prepend<T>(Typelist{}));
template <class T, class Typelist> using TLAppend = decltype(TLUtils::append<T>(Typelist{}));
template <template<class> class F, class Typelist>
using TLMap = decltype(TLUtils::map<F>(Typelist{}));
template <template<class...> class Root, class Typelist>
using TLChangeRoot = decltype(TLUtils::changeRoot<Root>(Typelist{}));

18
base/base/Typelists.h Normal file
View File

@ -0,0 +1,18 @@
#pragma once
#include "Typelist.h"
#include "extended_types.h"
#include "Decimal.h"
#include "UUID.h"
namespace DB
{
using TLIntegral = Typelist<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
using TLExtendedIntegral = Typelist<UInt128, Int128, UInt256, Int256>;
using TLDecimals = Typelist<Decimal32, Decimal64, Decimal128, Decimal256>;
using TLIntegralWithExtended = TLConcat<TLIntegral, TLExtendedIntegral>;
using TLNumbers = TLConcat<TLIntegralWithExtended, TLDecimals>;
using TLNumbersWithUUID = TLAppend<UUID, TLNumbers>;
}

9
base/base/UUID.h Normal file
View File

@ -0,0 +1,9 @@
#pragma once
#include "strong_typedef.h"
#include "extended_types.h"
namespace DB
{
using UUID = StrongTypedef<UInt128, struct UUIDTag>;
}

View File

@ -16,7 +16,7 @@ cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_C_CO
-DENABLE_LIBRARIES=0 -DENABLE_SSL=1 -DUSE_INTERNAL_SSL_LIBRARY=1 -DUSE_UNWIND=ON -DENABLE_EMBEDDED_COMPILER=0 \
-DENABLE_EXAMPLES=0 -DENABLE_UTILS=0 -DENABLE_THINLTO=0 "-DSANITIZE=$SANITIZER" \
-DENABLE_FUZZING=1 -DFUZZER='libfuzzer' -DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0 \
-DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" ..
-DENABLE_CHECK_HEAVY_BUILDS=1 -DGLIBC_COMPATIBILITY=OFF "${CMAKE_FLAGS[@]}" ..
FUZZER_TARGETS=$(find ../src -name '*_fuzzer.cpp' -execdir basename {} .cpp ';' | tr '\n' ' ')

View File

@ -82,7 +82,7 @@ Load from config:
```xml
<encryption_codecs>
<aes_128_gcm_siv>
<key>12345567812345678</key>
<key>1234567812345678</key>
</aes_128_gcm_siv>
</encryption_codecs>
```

View File

@ -16,5 +16,4 @@ The command changes the [sampling key](../../../engines/table-engines/mergetree-
The command is lightweight in the sense that it only changes metadata. The primary key must contain the new sample key.
!!! note "Note"
It only works for tables in the [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) family (including
[replicated](../../../engines/table-engines/mergetree-family/replication.md) tables).
It only works for tables in the [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) family (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) tables).

View File

@ -10,7 +10,7 @@ When data sampling is enabled, the query is not performed on all the data, but o
Approximated query processing can be useful in the following cases:
- When you have strict timing requirements (like \<100ms) but you cant justify the cost of additional hardware resources to meet them.
- When you have strict latency requirements (like below 100ms) but you cant justify the cost of additional hardware resources to meet them.
- When your raw data is not accurate, so approximation does not noticeably degrade the quality.
- Business requirements target approximate results (for cost-effectiveness, or to market exact results to premium users).

View File

@ -1116,8 +1116,6 @@ void Client::processOptions(const OptionsDescription & options_description,
if (options.count("config-file") && options.count("config"))
throw Exception("Two or more configuration files referenced in arguments", ErrorCodes::BAD_ARGUMENTS);
query_processing_stage = QueryProcessingStage::fromString(options["stage"].as<std::string>());
if (options.count("config"))
config().setString("config-file", options["config"].as<std::string>());
if (options.count("host") && !options["host"].defaulted())

View File

@ -63,24 +63,26 @@ namespace ErrorCodes
}
void LocalServer::processError(const String & query) const
void LocalServer::processError(const String &) const
{
if (ignore_error)
return;
if (is_interactive)
{
String message;
if (server_exception)
{
bool print_stack_trace = config().getBool("stacktrace", false);
fmt::print(stderr, "Error on processing query '{}':\n{}\n", query, getExceptionMessage(*server_exception, print_stack_trace, true));
fmt::print(stderr, "\n");
message = getExceptionMessage(*server_exception, print_stack_trace, true);
}
if (client_exception)
else if (client_exception)
{
fmt::print(stderr, "Error on processing query '{}':\n{}\n", query, client_exception->message());
fmt::print(stderr, "\n");
message = client_exception->message();
}
fmt::print(stderr, "Received exception:\n{}\n", message);
fmt::print(stderr, "\n");
}
else
{

View File

@ -33,6 +33,7 @@
:root {
--background-color: #DDF8FF; /* Or #FFFBEF; actually many pastel colors look great for light theme. */
--element-background-color: #FFF;
--bar-color: #F8F4F0; /* Light bar in background of table cells. */
--border-color: #EEE;
--shadow-color: rgba(0, 0, 0, 0.1);
--button-color: #FFAA00; /* Orange on light-cyan is especially good. */
@ -52,6 +53,7 @@
[data-theme="dark"] {
--background-color: #000;
--element-background-color: #102030;
--bar-color: #182838;
--border-color: #111;
--shadow-color: rgba(255, 255, 255, 0.1);
--text-color: #CCC;
@ -568,17 +570,20 @@
let thead = document.createElement('thead');
for (let idx in response.meta) {
let th = document.createElement('th');
let name = document.createTextNode(response.meta[idx].name);
const name = document.createTextNode(response.meta[idx].name);
th.appendChild(name);
thead.appendChild(th);
}
/// To prevent hanging the browser, limit the number of cells in a table.
/// It's important to have the limit on number of cells, not just rows, because tables may be wide or narrow.
let max_rows = 10000 / response.meta.length;
const max_rows = 10000 / response.meta.length;
let row_num = 0;
let column_classes = response.meta.map(elem => elem.type.match(/^(U?Int|Decimal|Float)/) ? 'right' : 'left');
const column_is_number = response.meta.map(elem => !!elem.type.match(/^(U?Int|Decimal|Float)/));
const column_maximums = column_is_number.map((elem, idx) => elem ? Math.max(...response.data.map(row => row[idx])) : 0);
const column_minimums = column_is_number.map((elem, idx) => elem ? Math.min(...response.data.map(row => Math.max(0, row[idx]))) : 0);
const column_need_render_bars = column_is_number.map((elem, idx) => column_maximums[idx] > 0 && column_maximums[idx] > column_minimums[idx]);
let tbody = document.createElement('tbody');
for (let row_idx in response.data) {
@ -614,11 +619,27 @@
node = link;
}
td.appendChild(node);
td.className = column_classes[col_idx];
td.className = column_is_number[col_idx] ? 'right' : 'left';
if (is_null) {
td.className += ' null';
}
/// If it's a number, render bar in background.
if (column_need_render_bars[col_idx] && text > 0) {
const ratio = 100 * text / column_maximums[col_idx];
let div = document.createElement('div');
div.style.width = '100%';
div.style.background = `linear-gradient(to right,
var(--bar-color) 0%, var(--bar-color) ${ratio}%,
transparent ${ratio}%, transparent 100%)`;
div.appendChild(node);
node = div;
}
td.appendChild(node);
tr.appendChild(td);
}
tbody.appendChild(tr);

View File

@ -3,9 +3,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <Core/TypeListNumber.h>
namespace DB
{
struct Settings;

View File

@ -1577,6 +1577,8 @@ void ClientBase::init(int argc, char ** argv)
if (options.count("log-level"))
Poco::Logger::root().setLevel(options["log-level"].as<std::string>());
query_processing_stage = QueryProcessingStage::fromString(options["stage"].as<std::string>());
processOptions(options_description, options, external_tables_arguments);
argsToConfig(common_arguments, config(), 100);
clearPasswordFromCommandLine(argc, argv);

View File

@ -8,7 +8,7 @@
#include <Common/typeid_cast.h>
#include <base/sort.h>
#include <Core/TypeId.h>
#include <Core/TypeName.h>
#include <base/TypeName.h>
#include <cmath>
@ -86,7 +86,7 @@ private:
{}
public:
const char * getFamilyName() const override { return TypeName<T>; }
const char * getFamilyName() const override { return TypeName<T>.data(); }
TypeIndex getDataType() const override { return TypeId<T>; }
bool isNumeric() const override { return false; }

View File

@ -8,7 +8,7 @@
#include <Core/Field.h>
#include <Common/assert_cast.h>
#include <Core/TypeId.h>
#include <Core/TypeName.h>
#include <base/TypeName.h>
namespace DB
@ -237,7 +237,7 @@ public:
data.reserve(n);
}
const char * getFamilyName() const override { return TypeName<T>; }
const char * getFamilyName() const override { return TypeName<T>.data(); }
TypeIndex getDataType() const override { return TypeId<T>; }
MutableColumnPtr cloneResized(size_t size) const override;

View File

@ -1,123 +0,0 @@
#pragma once
#include <cstddef>
#include <utility>
#include <Core/Defines.h>
namespace DB
{
template <typename... TTail>
struct TypeList
{
static constexpr size_t size = 0;
template <size_t I>
using At = std::nullptr_t;
template <typename Func, size_t index = 0>
static void forEach(Func && /*func*/)
{
}
};
template <typename THead, typename... TTail>
struct TypeList<THead, TTail...>
{
using Head = THead;
using Tail = TypeList<TTail...>;
static constexpr size_t size = 1 + sizeof...(TTail);
template <size_t I>
using At = typename std::template conditional_t<I == 0, Head, typename Tail::template At<I - 1>>;
template <typename Func, size_t index = 0>
static void ALWAYS_INLINE forEach(Func && func)
{
func.template operator()<Head, index>();
Tail::template forEach<Func, index + 1>(std::forward<Func>(func));
}
};
/// Prepend Type to TypeList
/// Usage:
/// using TypeListWithType = typename AppendToTypeList<Type, ConcreteTypeList>::Type;
template <typename TypeToPrepend, typename List, typename ... Types>
struct PrependToTypeList
{
using Type = typename PrependToTypeList<TypeToPrepend, typename List::Tail, Types ..., typename List::Head>::Type;
};
template <typename TypeToPrepend, typename ... Types>
struct PrependToTypeList<TypeToPrepend, TypeList<>, Types ...>
{
using Type = TypeList<TypeToPrepend, Types ...>;
};
/// Append Type to TypeList
/// Usage:
/// using TypeListWithType = typename AppendToTypeList<Type, ConcreteTypeList>::Type;
template <typename TypeToAppend, typename List, typename ... Types>
struct AppendToTypeList
{
using Type = typename AppendToTypeList<TypeToAppend, typename List::Tail, Types ..., typename List::Head>::Type;
};
template <typename TypeToAppend, typename ... Types>
struct AppendToTypeList<TypeToAppend, TypeList<>, Types ...>
{
using Type = TypeList<Types ..., TypeToAppend>;
};
/// Apply TypeList as variadic template argument of Class.
/// Usage:
/// using ClassWithAppliedTypeList = typename ApplyTypeListForClass<Class, ConcreteTypeList>::Type;
template <template <typename ...> typename Class, typename List, typename ... Types>
struct ApplyTypeListForClass
{
using Type = typename ApplyTypeListForClass<Class, typename List::Tail, Types ..., typename List::Head>::Type;
};
template <template <typename ...> typename Class, typename ... Types>
struct ApplyTypeListForClass<Class, TypeList<>, Types ...>
{
using Type = Class<Types ...>;
};
/// TypeList concatenation.
/// Usage:
/// using ResultTypeList = typename TypeListConcat<LeftList, RightList>::Type;
template <typename TypeListLeft, typename TypeListRight>
struct TypeListConcat
{
using Type = typename TypeListConcat<
typename AppendToTypeList<typename TypeListRight::Head, TypeListLeft>::Type,
typename TypeListRight::Tail>::Type;
};
template <typename TypeListLeft>
struct TypeListConcat<TypeListLeft, TypeList<>>
{
using Type = TypeListLeft;
};
/// TypeList Map function.
/// Usage:
/// using ResultTypeList = typename TypeListMap<Function, TypeListArgs>::Type;
template <template <typename> typename Function, typename TypeListArgs>
struct TypeListMap
{
using Type = typename PrependToTypeList<
Function<typename TypeListArgs::Head>,
typename TypeListMap<Function, typename TypeListArgs::Tail>::Type>::Type;
};
template <template <typename> typename Function>
struct TypeListMap<Function, TypeList<>>
{
using Type = TypeList<>;
};
}

View File

@ -1,7 +1,7 @@
#pragma once
#include <base/demangle.h>
#include <Common/TypeList.h>
#include <base/Typelist.h>
#include <Common/Exception.h>
/* Generic utils which are intended for visitor pattern implementation.
@ -64,7 +64,7 @@ template <>
class Visitor<>
{
public:
using List = TypeList<>;
using List = Typelist<>;
protected:
~Visitor() = default;
@ -74,7 +74,7 @@ template <typename Type>
class Visitor<Type> : public Visitor<>
{
public:
using List = TypeList<Type>;
using List = Typelist<Type>;
virtual void visit(Type &) = 0;
@ -86,7 +86,7 @@ template <typename Type, typename ... Types>
class Visitor<Type, Types ...> : public Visitor<Types ...>
{
public:
using List = TypeList<Type, Types ...>;
using List = Typelist<Type, Types ...>;
using Visitor<Types ...>::visit;
virtual void visit(Type &) = 0;
@ -145,13 +145,13 @@ protected:
template <typename Derived, typename VisitorBase>
class VisitorImpl : public
ApplyTypeListForClass<
VisitorImplHelper,
typename TypeListConcat<
TypeList<Derived, VisitorBase>,
typename VisitorBase::List
>::Type
>::Type
TLChangeRoot<
VisitorImplHelper,
TLConcat<
Typelist<Derived, VisitorBase>,
typename VisitorBase::List
>
>
{
protected:
~VisitorImpl() = default;

View File

@ -5,9 +5,9 @@
namespace DB
{
String quoteString(const StringRef & x)
String quoteString(std::string_view x)
{
String res(x.size, '\0');
String res(x.size(), '\0');
WriteBufferFromString wb(res);
writeQuotedString(x, wb);
return res;

View File

@ -2,12 +2,18 @@
#include <base/types.h>
#include <base/StringRef.h>
#include <concepts>
namespace DB
{
/// Quote the string.
String quoteString(const StringRef & x);
[[nodiscard]] String quoteString(std::string_view x);
// Prefer string_view over StringRef for implicit conversions
[[nodiscard]] inline String quoteString(std::same_as<StringRef> auto x)
{
return quoteString(std::string_view{x.data, x.size});
}
/// Double quote the string.
String doubleQuoteString(const StringRef & x);

View File

@ -315,8 +315,19 @@ void CompressionCodecEncrypted::Configuration::loadImpl(
if (new_params->keys_storage[method].empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No keys, an encryption needs keys to work");
/// Try to find which key will be used for encryption. If there is no current_key,
/// first key will be used for encryption (its index equals to zero).
if (!config.has(config_prefix + ".current_key_id"))
{
/// In case of multiple keys, current_key_id is mandatory
if (new_params->keys_storage[method].size() > 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There are multiple keys in config. current_key_id is required");
/// If there is only one key with non zero ID, curren_key_id should be defined.
if (new_params->keys_storage[method].size() == 1 && !new_params->keys_storage[method].contains(0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config has one key with non zero id. сurrent_key_id is required");
}
/// Try to find which key will be used for encryption. If there is no current_key and only one key without id
/// or with zero id, first key will be used for encryption (its index equals to zero).
new_params->current_key_id[method] = config.getUInt64(config_prefix + ".current_key_id", 0);
/// Check that we have current key. Otherwise config is incorrect.

View File

@ -1,18 +0,0 @@
#pragma once
#include <Core/Types.h>
#include <Common/TypeList.h>
namespace DB
{
using TypeListNativeNumbers = TypeList<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
using TypeListExtendedNumbers = TypeList<UInt128, Int128, UInt256, Int256>;
using TypeListDecimalNumbers = TypeList<Decimal32, Decimal64, Decimal128, Decimal256>;
using TypeListGeneralNumbers = typename TypeListConcat<TypeListNativeNumbers, TypeListExtendedNumbers>::Type;
using TypeListNumbers = typename TypeListConcat<TypeListGeneralNumbers, TypeListDecimalNumbers>::Type;
using TypeListNumbersAndUUID = typename TypeListConcat<TypeListNumbers, TypeList<UUID>>::Type;
}

View File

@ -1,39 +0,0 @@
#pragma once
#include <Core/Types.h>
namespace DB
{
struct Array;
struct Tuple;
struct Map;
struct AggregateFunctionStateData;
template <class T> constexpr const char * TypeName = "";
template <> inline constexpr const char * TypeName<UInt8> = "UInt8";
template <> inline constexpr const char * TypeName<UInt16> = "UInt16";
template <> inline constexpr const char * TypeName<UInt32> = "UInt32";
template <> inline constexpr const char * TypeName<UInt64> = "UInt64";
template <> inline constexpr const char * TypeName<UInt128> = "UInt128";
template <> inline constexpr const char * TypeName<UInt256> = "UInt256";
template <> inline constexpr const char * TypeName<Int8> = "Int8";
template <> inline constexpr const char * TypeName<Int16> = "Int16";
template <> inline constexpr const char * TypeName<Int32> = "Int32";
template <> inline constexpr const char * TypeName<Int64> = "Int64";
template <> inline constexpr const char * TypeName<Int128> = "Int128";
template <> inline constexpr const char * TypeName<Int256> = "Int256";
template <> inline constexpr const char * TypeName<Float32> = "Float32";
template <> inline constexpr const char * TypeName<Float64> = "Float64";
template <> inline constexpr const char * TypeName<String> = "String";
template <> inline constexpr const char * TypeName<UUID> = "UUID";
template <> inline constexpr const char * TypeName<Decimal32> = "Decimal32";
template <> inline constexpr const char * TypeName<Decimal64> = "Decimal64";
template <> inline constexpr const char * TypeName<Decimal128> = "Decimal128";
template <> inline constexpr const char * TypeName<Decimal256> = "Decimal256";
template <> inline constexpr const char * TypeName<DateTime64> = "DateTime64";
template <> inline constexpr const char * TypeName<Array> = "Array";
template <> inline constexpr const char * TypeName<Tuple> = "Tuple";
template <> inline constexpr const char * TypeName<Map> = "Map";
template <> inline constexpr const char * TypeName<AggregateFunctionStateData> = "AggregateFunctionState";
}

View File

@ -6,6 +6,7 @@
#include <base/strong_typedef.h>
#include <base/Decimal.h>
#include <base/defines.h>
#include <base/UUID.h>
namespace DB
@ -97,8 +98,6 @@ using UInt256 = ::UInt256;
using Int128 = ::Int128;
using Int256 = ::Int256;
STRONG_TYPEDEF(UInt128, UUID)
/// Not a data type in database, defined just for convenience.
using Strings = std::vector<String>;
}

View File

@ -6,7 +6,7 @@
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include <Core/Field.h>
#include <Core/TypeListNumber.h>
#include <base/Typelists.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeFactory.h>
@ -53,8 +53,8 @@ namespace
{
}
template <typename T, size_t>
void operator()()
template <class T>
void operator()(Id<T>)
{
if (typeid_cast<const DataTypeNumber<T> *>(&keys_type))
column = creator(static_cast<ColumnVector<T> *>(nullptr));
@ -89,7 +89,7 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat
else if (which.isInt() || which.isUInt() || which.isFloat())
{
MutableColumnUniquePtr column;
TypeListNativeNumbers::forEach(CreateColumnVector(column, *type, creator));
TLUtils::forEach(TLIntegral{}, CreateColumnVector(column, *type, creator));
if (!column)
throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR);

View File

@ -24,7 +24,7 @@ public:
using FieldType = T;
using ColumnType = ColumnVector<T>;
const char * getFamilyName() const override { return TypeName<T>; }
const char * getFamilyName() const override { return TypeName<T>.data(); }
TypeIndex getTypeId() const override { return TypeId<T>; }
Field getDefault() const override;

View File

@ -1,7 +1,7 @@
#include "RangeHashedDictionary.h"
#include <Columns/ColumnNullable.h>
#include <Functions/FunctionHelpers.h>
#include <Common/TypeList.h>
#include <base/Typelists.h>
#include <Interpreters/castColumn.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Dictionaries/DictionaryFactory.h>
@ -650,8 +650,8 @@ struct RangeHashedDictionaryCallGetSourceImpl
const Names * column_names;
size_t max_block_size;
template <typename RangeType, size_t>
void operator()()
template <class RangeType>
void operator()(Id<RangeType>)
{
const auto & type = dict->dict_struct.range_min->type;
if (pipe.empty() && dynamic_cast<const DataTypeNumberBase<RangeType> *>(type.get()))
@ -662,14 +662,12 @@ struct RangeHashedDictionaryCallGetSourceImpl
template <DictionaryKeyType dictionary_key_type>
Pipe RangeHashedDictionary<dictionary_key_type>::read(const Names & column_names, size_t max_block_size) const
{
using ListType = TypeList<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Int128, Float32, Float64>;
RangeHashedDictionaryCallGetSourceImpl<dictionary_key_type> callable;
callable.dict = this;
callable.column_names = &column_names;
callable.max_block_size = max_block_size;
ListType::forEach(callable);
TLUtils::forEach(TLIntegral{}, callable);
if (callable.pipe.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR,

View File

@ -76,6 +76,17 @@ namespace
if (res->keys.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No keys, an encrypted disk needs keys to work");
if (!config.has(config_prefix + ".current_key_id"))
{
/// In case of multiple keys, current_key_id is mandatory
if (res->keys.size() > 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There are multiple keys in config. current_key_id is required");
/// If there is only one key with non zero ID, curren_key_id should be defined.
if (res->keys.size() == 1 && !res->keys.contains(0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config has one key with non zero id. сurrent_key_id is required");
}
res->current_key_id = config.getUInt64(config_prefix + ".current_key_id", 0);
if (!res->keys.contains(res->current_key_id))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found a key with the current ID {}", res->current_key_id);

View File

@ -35,7 +35,7 @@
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/TypeList.h>
#include <base/Typelist.h>
#include <base/map.h>
#if !defined(ARCADIA_BUILD)
@ -577,20 +577,20 @@ class FunctionBinaryArithmetic : public IFunction
static bool castType(const IDataType * type, auto && f)
{
using Types = TypeList<
using Types = Typelist<
DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64, DataTypeUInt128, DataTypeUInt256,
DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256,
DataTypeDecimal32, DataTypeDecimal64, DataTypeDecimal128, DataTypeDecimal256,
DataTypeDate, DataTypeDateTime,
DataTypeFixedString, DataTypeString>;
using Floats = TypeList<DataTypeFloat32, DataTypeFloat64>;
using Floats = Typelist<DataTypeFloat32, DataTypeFloat64>;
using ValidTypes = std::conditional_t<valid_on_float_arguments,
typename TypeListConcat<Types, Floats>::Type,
TLConcat<Types, Floats>,
Types>;
return castTypeToEitherTL<ValidTypes>(type, std::forward<decltype(f)>(f));
return castTypeToEither(ValidTypes{}, type, std::forward<decltype(f)>(f));
}
template <typename F>

View File

@ -304,19 +304,32 @@ public:
}
private:
template <class Parser>
ColumnPtr
chooseAndRunJSONParserOne(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
{
/// Only implementations with prepare() can handle NULL.
///
/// (and right now this file is pretty complex already, and adding
/// support of Nullable for others will make it even more complex)
if (null_presence.has_nullable && !Impl<Parser>::supportNullable())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} does not support Nullable", Name::name);
return FunctionJSONHelpers::Executor<Name, Impl, Parser>::run(arguments, result_type, input_rows_count);
}
ColumnPtr
chooseAndRunJSONParser(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
{
#if USE_SIMDJSON
if (allow_simdjson)
return FunctionJSONHelpers::Executor<Name, Impl, SimdJSONParser>::run(arguments, result_type, input_rows_count);
return chooseAndRunJSONParserOne<SimdJSONParser>(arguments, result_type, input_rows_count);
#endif
#if USE_RAPIDJSON
return FunctionJSONHelpers::Executor<Name, Impl, RapidJSONParser>::run(arguments, result_type, input_rows_count);
return chooseAndRunJSONParserOne<RapidJSONParser>(arguments, result_type, input_rows_count);
#else
return FunctionJSONHelpers::Executor<Name, Impl, DummyJSONParser>::run(arguments, result_type, input_rows_count);
return chooseAndRunJSONParserOne<DummyJSONParser>(arguments, result_type, input_rows_count);
#endif
}
@ -433,6 +446,7 @@ public:
static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared<DataTypeUInt8>(); }
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool supportNullable() { return false; }
static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &)
{
@ -461,6 +475,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &) { return 0; }
static bool supportNullable() { return false; }
static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &)
{
@ -485,6 +500,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool supportNullable() { return false; }
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
{
@ -515,6 +531,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool supportNullable() { return false; }
static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view & last_key)
{
@ -549,6 +566,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool supportNullable() { return false; }
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
{
@ -591,6 +609,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool supportNullable() { return false; }
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
{
@ -666,6 +685,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool supportNullable() { return false; }
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
{
@ -691,6 +711,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool supportNullable() { return false; }
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
{
@ -1090,6 +1111,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; }
static bool supportNullable() { return true; }
void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type)
{
@ -1131,6 +1153,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; }
static bool supportNullable() { return true; }
void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type)
{
@ -1182,6 +1205,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool supportNullable() { return false; }
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
{
@ -1286,6 +1310,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool supportNullable() { return false; }
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
{
@ -1318,6 +1343,7 @@ public:
}
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool supportNullable() { return false; }
bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
{

View File

@ -1,6 +1,6 @@
#pragma once
#include <Common/Visitor.h>
#include <Core/TypeListNumber.h>
#include <base/Typelists.h>
namespace DB::GatherUtils
{
@ -14,12 +14,12 @@ struct GenericArraySink;
template <typename ArraySink>
struct NullableArraySink;
using NumericArraySinks = typename TypeListMap<NumericArraySink, TypeListNumbersAndUUID>::Type;
using BasicArraySinks = typename AppendToTypeList<GenericArraySink, NumericArraySinks>::Type;
using NullableArraySinks = typename TypeListMap<NullableArraySink, BasicArraySinks>::Type;
using TypeListArraySinks = typename TypeListConcat<BasicArraySinks, NullableArraySinks>::Type;
using NumericArraySinks = TLMap<NumericArraySink, TLNumbersWithUUID>;
using BasicArraySinks = TLAppend<GenericArraySink, NumericArraySinks>;
using NullableArraySinks = TLMap<NullableArraySink, BasicArraySinks>;
using TLArraySinks = TLConcat<BasicArraySinks, NullableArraySinks>;
class ArraySinkVisitor : public ApplyTypeListForClass<Visitor, TypeListArraySinks>::Type
class ArraySinkVisitor : public TLChangeRoot<Visitor, TLArraySinks>
{
protected:
~ArraySinkVisitor() = default;

View File

@ -1,6 +1,6 @@
#pragma once
#include <Common/Visitor.h>
#include <Core/TypeListNumber.h>
#include <base/Typelists.h>
namespace DB::GatherUtils
{
@ -17,10 +17,10 @@ struct NullableArraySource;
template <typename Base>
struct ConstSource;
using NumericArraySources = typename TypeListMap<NumericArraySource, TypeListNumbersAndUUID>::Type;
using BasicArraySources = typename AppendToTypeList<GenericArraySource, NumericArraySources>::Type;
using NumericArraySources = TLMap<NumericArraySource, TLNumbersWithUUID>;
using BasicArraySources = TLAppend<GenericArraySource, NumericArraySources>;
class ArraySourceVisitor : public ApplyTypeListForClass<Visitor, BasicArraySources>::Type
class ArraySourceVisitor : public TLChangeRoot<Visitor, BasicArraySources>
{
protected:
~ArraySourceVisitor() = default;

View File

@ -1,6 +1,6 @@
#pragma once
#include <Common/Visitor.h>
#include <Core/TypeListNumber.h>
#include <base/Typelists.h>
namespace DB::GatherUtils
{
@ -17,14 +17,14 @@ struct NullableValueSource;
template <typename Base>
struct ConstSource;
using NumericValueSources = typename TypeListMap<NumericValueSource, TypeListNumbersAndUUID>::Type;
using BasicValueSources = typename AppendToTypeList<GenericValueSource, NumericValueSources>::Type;
using NullableValueSources = typename TypeListMap<NullableValueSource, BasicValueSources>::Type;
using BasicAndNullableValueSources = typename TypeListConcat<BasicValueSources, NullableValueSources>::Type;
using ConstValueSources = typename TypeListMap<ConstSource, BasicAndNullableValueSources>::Type;
using TypeListValueSources = typename TypeListConcat<BasicAndNullableValueSources, ConstValueSources>::Type;
using NumericValueSources = TLMap<NumericValueSource, TLNumbersWithUUID>;
using BasicValueSources = TLAppend<GenericValueSource, NumericValueSources>;
using NullableValueSources = TLMap<NullableValueSource, BasicValueSources>;
using BasicAndNullableValueSources = TLConcat<BasicValueSources, NullableValueSources>;
using ConstValueSources = TLMap<ConstSource, BasicAndNullableValueSources>;
using TypeListValueSources = TLConcat<BasicAndNullableValueSources, ConstValueSources>;
class ValueSourceVisitor : public ApplyTypeListForClass<Visitor, TypeListValueSources>::Type
class ValueSourceVisitor : public TLChangeRoot<Visitor, TypeListValueSources>
{
protected:
~ValueSourceVisitor() = default;

View File

@ -1,7 +1,7 @@
#include "GatherUtils.h"
#include "Sinks.h"
#include "Sources.h"
#include <Core/TypeListNumber.h>
#include <base/Typelists.h>
namespace DB::GatherUtils
{
@ -55,7 +55,7 @@ struct ArraySinkCreator<>
std::unique_ptr<IArraySink> createArraySink(ColumnArray & col, size_t column_size)
{
using Creator = ApplyTypeListForClass<ArraySinkCreator, TypeListNumbersAndUUID>::Type;
using Creator = TLChangeRoot<ArraySinkCreator, TLNumbersWithUUID>;
return Creator::create(col.getData(), col.getOffsets(), column_size);
}
}

View File

@ -1,7 +1,7 @@
#include "GatherUtils.h"
#include "Sinks.h"
#include "Sources.h"
#include <Core/TypeListNumber.h>
#include <base/Typelists.h>
namespace DB::GatherUtils
{
@ -58,7 +58,7 @@ struct ArraySourceCreator<>
std::unique_ptr<IArraySource> createArraySource(const ColumnArray & col, bool is_const, size_t total_rows)
{
using Creator = typename ApplyTypeListForClass<ArraySourceCreator, TypeListNumbersAndUUID>::Type;
using Creator = TLChangeRoot<ArraySourceCreator, TLNumbersWithUUID>;
if (const auto * column_nullable = typeid_cast<const ColumnNullable *>(&col.getData()))
{
auto column = ColumnArray::create(column_nullable->getNestedColumnPtr(), col.getOffsetsPtr());

View File

@ -1,7 +1,7 @@
#include "GatherUtils.h"
#include "Sinks.h"
#include "Sources.h"
#include <Core/TypeListNumber.h>
#include <base/Typelists.h>
namespace DB::GatherUtils
{
@ -58,7 +58,7 @@ struct ValueSourceCreator<>
std::unique_ptr<IValueSource> createValueSource(const IColumn & col, bool is_const, size_t total_rows)
{
using Creator = typename ApplyTypeListForClass<ValueSourceCreator, TypeListNumbersAndUUID>::Type;
using Creator = TLChangeRoot<ValueSourceCreator, TLNumbersWithUUID>;
if (const auto * column_nullable = typeid_cast<const ColumnNullable *>(&col))
{
return Creator::create(column_nullable->getNestedColumn(), &column_nullable->getNullMapData(), is_const, total_rows);

View File

@ -2,7 +2,7 @@
#include <base/types.h>
#include <Core/Defines.h>
#include <Core/TypeListNumber.h>
#include <base/Typelists.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnVector.h>
#include <Common/typeid_cast.h>
@ -604,7 +604,7 @@ struct CallPointInPolygon<Type, Types ...>
template <typename PointInPolygonImpl>
static ColumnPtr call(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
{
using Impl = typename ApplyTypeListForClass<CallPointInPolygon, TypeListNativeNumbers>::Type;
using Impl = TLChangeRoot<CallPointInPolygon, TLIntegral>;
if (auto column = typeid_cast<const ColumnVector<Type> *>(&x))
return Impl::template call<Type>(*column, y, impl);
return CallPointInPolygon<Types ...>::call(x, y, impl);
@ -630,7 +630,7 @@ struct CallPointInPolygon<>
template <typename PointInPolygonImpl>
NO_INLINE ColumnPtr pointInPolygon(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
{
using Impl = typename ApplyTypeListForClass<CallPointInPolygon, TypeListNativeNumbers>::Type;
using Impl = TLChangeRoot<CallPointInPolygon, TLIntegral>;
return Impl::call(x, y, impl);
}

View File

@ -20,7 +20,7 @@
#include <Columns/ColumnTuple.h>
#include <Common/HashTable/ClearableHashMap.h>
#include <Common/assert_cast.h>
#include <Core/TypeListNumber.h>
#include <base/Typelists.h>
#include <Interpreters/castColumn.h>
#include <base/range.h>
@ -106,8 +106,8 @@ private:
NumberExecutor(const UnpackedArrays & arrays_, const DataTypePtr & data_type_, ColumnPtr & result_)
: arrays(arrays_), data_type(data_type_), result(result_) {}
template <typename T, size_t>
void operator()();
template <class T>
void operator()(Id<T>);
};
struct DecimalExecutor
@ -119,8 +119,8 @@ private:
DecimalExecutor(const UnpackedArrays & arrays_, const DataTypePtr & data_type_, ColumnPtr & result_)
: arrays(arrays_), data_type(data_type_), result(result_) {}
template <typename T, size_t>
void operator()();
template <class T>
void operator()(Id<T>);
};
};
@ -403,8 +403,8 @@ ColumnPtr FunctionArrayIntersect::executeImpl(const ColumnsWithTypeAndName & arg
ColumnPtr result_column;
auto not_nullable_nested_return_type = removeNullable(nested_return_type);
TypeListNativeNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column));
TypeListDecimalNumbers::forEach(DecimalExecutor(arrays, not_nullable_nested_return_type, result_column));
TLUtils::forEach(TLIntegral{}, NumberExecutor(arrays, not_nullable_nested_return_type, result_column));
TLUtils::forEach(TLDecimals{}, DecimalExecutor(arrays, not_nullable_nested_return_type, result_column));
using DateMap = ClearableHashMapWithStackMemory<DataTypeDate::FieldType,
size_t, DefaultHash<DataTypeDate::FieldType>, INITIAL_SIZE_DEGREE>;
@ -444,8 +444,8 @@ ColumnPtr FunctionArrayIntersect::executeImpl(const ColumnsWithTypeAndName & arg
return result_column;
}
template <typename T, size_t>
void FunctionArrayIntersect::NumberExecutor::operator()()
template <class T>
void FunctionArrayIntersect::NumberExecutor::operator()(Id<T>)
{
using Container = ClearableHashMapWithStackMemory<T, size_t, DefaultHash<T>,
INITIAL_SIZE_DEGREE>;
@ -454,8 +454,8 @@ void FunctionArrayIntersect::NumberExecutor::operator()()
result = execute<Container, ColumnVector<T>, true>(arrays, ColumnVector<T>::create());
}
template <typename T, size_t>
void FunctionArrayIntersect::DecimalExecutor::operator()()
template <class T>
void FunctionArrayIntersect::DecimalExecutor::operator()(Id<T>)
{
using Container = ClearableHashMapWithStackMemory<T, size_t, DefaultHash<T>,
INITIAL_SIZE_DEGREE>;

View File

@ -1,24 +1,10 @@
#pragma once
#include <Common/typeid_cast.h>
#include <base/Typelist.h>
namespace DB
{
namespace detail
{
template <typename Typelist, size_t ...I>
static bool castTypeToEither(const auto * type, auto && f, std::index_sequence<I...>)
{
return (
(typeid_cast<const typename Typelist::template At<I> *>(type)
? std::forward<decltype(f)>(f)(
*typeid_cast<const typename Typelist::template At<I> *>(type))
: false)
|| ...);
}
}
template <typename... Ts, typename T, typename F>
static bool castTypeToEither(const T * type, F && f)
{
@ -26,12 +12,14 @@ static bool castTypeToEither(const T * type, F && f)
return ((typeid_cast<const Ts *>(type) ? f(*typeid_cast<const Ts *>(type)) : false) || ...);
}
/// Use Common/TypeList as template argument
template <class Typelist>
static constexpr bool castTypeToEitherTL(const auto * type, auto && f)
template <class ...Args>
constexpr bool castTypeToEither(Typelist<Args...>, const auto * type, auto && f)
{
return detail::castTypeToEither<Typelist>(
type, std::forward<decltype(f)>(f),
std::make_index_sequence<Typelist::size>());
return (
(typeid_cast<const Args *>(type) != nullptr
? std::forward<decltype(f)>(f)(
*typeid_cast<const Args *>(type))
: false)
|| ...);
}
}

View File

@ -67,9 +67,11 @@ public:
const auto * type_col = checkAndGetColumnConst<ColumnString>(column.get());
if (!type_col)
throw Exception("Second argument to " + getName() + " must be a constant string describing type."
" Instead there is non-constant column of type " + arguments.back().type->getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument to {} must be a constant string describing type."
" Instead there is non-constant column of type {}",
getName(),
arguments.back().type->getName());
DataTypePtr to_type = DataTypeFactory::instance().get(type_col->getValue<String>());
@ -78,30 +80,34 @@ public:
if (result_reinterpret_type.isFixedString())
{
if (!from_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion())
throw Exception("Cannot reinterpret " + from_type->getName() +
" as FixedString because it is not fixed size and contiguous in memory",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Cannot reinterpret {} as FixedString because it is not fixed size and contiguous in memory",
from_type->getName());
}
else if (result_reinterpret_type.isString())
{
if (!from_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
throw Exception("Cannot reinterpret " + from_type->getName() +
" as String because it is not contiguous in memory",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Cannot reinterpret {} as String because it is not contiguous in memory",
from_type->getName());
}
else if (canBeReinterpretedAsNumeric(result_reinterpret_type))
{
WhichDataType from_data_type(from_type);
if (!canBeReinterpretedAsNumeric(from_data_type) && !from_data_type.isStringOrFixedString())
throw Exception("Cannot reinterpret " + from_type->getName() + " as " + to_type->getName()
+ " because only Numeric, String or FixedString can be reinterpreted in Numeric",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Cannot reinterpret {} as {} because only Numeric, String or FixedString can be reinterpreted in Numeric",
from_type->getName(),
to_type->getName());
}
else
throw Exception("Cannot reinterpret " + from_type->getName() + " as " + to_type->getName()
+ " because only reinterpretation in String, FixedString and Numeric types is supported",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Cannot reinterpret {} as {} because only reinterpretation in String, FixedString and Numeric types is supported",
from_type->getName(),
to_type->getName());
}
return to_type;
}
@ -156,7 +162,6 @@ public:
}
else if constexpr (CanBeReinterpretedAsNumeric<ToType>)
{
using ToColumnType = typename ToType::ColumnType;
using ToFieldType = typename ToType::FieldType;
if constexpr (std::is_same_v<FromType, DataTypeString>)
@ -165,10 +170,10 @@ public:
auto col_res = numericColumnCreateHelper<ToType>(static_cast<const ToType&>(*result_type.get()));
const ColumnString::Chars & data_from = col_from->getChars();
const ColumnString::Offsets & offsets_from = col_from->getOffsets();
const auto & data_from = col_from->getChars();
const auto & offsets_from = col_from->getOffsets();
size_t size = offsets_from.size();
typename ToColumnType::Container & vec_res = col_res->getData();
auto & vec_res = col_res->getData();
vec_res.resize(size);
size_t offset = 0;
@ -192,10 +197,10 @@ public:
auto col_res = numericColumnCreateHelper<ToType>(static_cast<const ToType&>(*result_type.get()));
const ColumnString::Chars & data_from = col_from_fixed->getChars();
const auto& data_from = col_from_fixed->getChars();
size_t step = col_from_fixed->getN();
size_t size = data_from.size() / step;
typename ToColumnType::Container & vec_res = col_res->getData();
auto & vec_res = col_res->getData();
vec_res.resize(size);
size_t offset = 0;
@ -243,8 +248,10 @@ public:
return false;
}))
{
throw Exception("Cannot reinterpret " + from_type->getName() + " as " + result_type->getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Cannot reinterpret {} as {}",
from_type->getName(),
result_type->getName());
}
return result;
@ -255,6 +262,7 @@ private:
IsDataTypeDecimalOrNumber<T> ||
std::is_same_v<T, DataTypeDate> ||
std::is_same_v<T, DataTypeDateTime> ||
std::is_same_v<T, DataTypeDateTime64> ||
std::is_same_v<T, DataTypeUUID>;
static bool canBeReinterpretedAsNumeric(const WhichDataType & type)
@ -359,9 +367,12 @@ public:
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForConstants() const override { return impl.useDefaultImplementationForConstants(); }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override
{
return impl.isSuitableForShortCircuitArgumentsExecution(arguments);
}
static ColumnsWithTypeAndName addTypeColumnToArguments(const ColumnsWithTypeAndName & arguments)
{
@ -374,9 +385,9 @@ public:
const auto & type = argument.type;
if (!type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion())
throw Exception("Cannot reinterpret " + type->getName() +
" as FixedString because it is not fixed size and contiguous in memory",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Cannot reinterpret {} as FixedString because it is not fixed size and contiguous in memory",
type->getName());
size_t type_value_size_in_memory = type->getSizeOfValueInMemory();
data_type = std::make_shared<DataTypeFixedString>(type_value_size_in_memory);

View File

@ -4,4 +4,8 @@ endif()
if (ENABLE_EXAMPLES)
add_subdirectory(examples)
endif()
endif()
if (ENABLE_FUZZING)
add_subdirectory(fuzzers)
endif()

View File

@ -250,6 +250,61 @@ using CustomizeAggregateFunctionsOrNullVisitor = InDepthNodeVisitor<OneTypeMatch
using CustomizeAggregateFunctionsMoveOrNullVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeAggregateFunctionsMoveSuffixData>, true>;
using FuseSumCountAggregatesVisitor = InDepthNodeVisitor<OneTypeMatcher<FuseSumCountAggregatesVisitorData>, true>;
struct ExistsExpressionData
{
using TypeToVisit = ASTFunction;
static void visit(ASTFunction & func, ASTPtr)
{
bool exists_expression = func.name == "exists"
&& func.arguments && func.arguments->children.size() == 1
&& typeid_cast<const ASTSubquery *>(func.arguments->children[0].get());
if (!exists_expression)
return;
/// EXISTS(subquery) --> 1 IN (SELECT 1 FROM subquery LIMIT 1)
auto subquery_node = func.arguments->children[0];
auto table_expression = std::make_shared<ASTTableExpression>();
table_expression->subquery = std::move(subquery_node);
table_expression->children.push_back(table_expression->subquery);
auto tables_in_select_element = std::make_shared<ASTTablesInSelectQueryElement>();
tables_in_select_element->table_expression = std::move(table_expression);
tables_in_select_element->children.push_back(tables_in_select_element->table_expression);
auto tables_in_select = std::make_shared<ASTTablesInSelectQuery>();
tables_in_select->children.push_back(std::move(tables_in_select_element));
auto select_expr_list = std::make_shared<ASTExpressionList>();
select_expr_list->children.push_back(std::make_shared<ASTLiteral>(1u));
auto select_query = std::make_shared<ASTSelectQuery>();
select_query->children.push_back(select_expr_list);
select_query->setExpression(ASTSelectQuery::Expression::SELECT, select_expr_list);
select_query->setExpression(ASTSelectQuery::Expression::TABLES, tables_in_select);
ASTPtr limit_length_ast = std::make_shared<ASTLiteral>(Field(UInt64(1)));
select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(limit_length_ast));
auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
select_with_union_query->list_of_selects = std::make_shared<ASTExpressionList>();
select_with_union_query->list_of_selects->children.push_back(std::move(select_query));
select_with_union_query->children.push_back(select_with_union_query->list_of_selects);
auto new_subquery = std::make_shared<ASTSubquery>();
new_subquery->children.push_back(select_with_union_query);
auto function = makeASTFunction("in", std::make_shared<ASTLiteral>(1u), new_subquery);
func = *function;
}
};
using ExistsExpressionVisitor = InDepthNodeVisitor<OneTypeMatcher<ExistsExpressionData>, false>;
/// Translate qualified names such as db.table.column, table.column, table_alias.column to names' normal form.
/// Expand asterisks and qualified asterisks with column names.
/// There would be columns in normal form & column aliases after translation. Column & column alias would be normalized in QueryNormalizer.
@ -1091,6 +1146,9 @@ void TreeRewriter::normalize(
CustomizeIfDistinctVisitor::Data data_distinct_if{"DistinctIf"};
CustomizeIfDistinctVisitor(data_distinct_if).visit(query);
ExistsExpressionVisitor::Data exists;
ExistsExpressionVisitor(exists).visit(query);
if (settings.transform_null_in)
{
CustomizeInVisitor::Data data_null_in{"nullIn"};

View File

@ -0,0 +1,10 @@
add_executable(execute_query_fuzzer execute_query_fuzzer.cpp)
target_link_libraries(execute_query_fuzzer PRIVATE
dbms
clickhouse_functions
clickhouse_table_functions
clickhouse_aggregate_functions
clickhouse_dictionaries
clickhouse_dictionaries_embedded
${LIB_FUZZING_ENGINE})

View File

@ -0,0 +1,56 @@
#include <iostream>
#include <Interpreters/executeQuery.h>
#include <Interpreters/Context.h>
#include "Processors/Executors/PullingPipelineExecutor.h"
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
#include <Storages/registerStorages.h>
#include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h>
#include <Formats/registerFormats.h>
using namespace DB;
extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
try
{
std::string input = std::string(reinterpret_cast<const char*>(data), size);
static SharedContextHolder shared_context;
static ContextMutablePtr context;
auto initialize = [&]() mutable
{
shared_context = Context::createShared();
context = Context::createGlobal(shared_context.get());
context->makeGlobalContext();
context->setApplicationType(Context::ApplicationType::LOCAL);
registerFunctions();
registerAggregateFunctions();
registerTableFunctions();
registerStorages();
registerDictionaries();
registerDisks();
registerFormats();
return true;
};
static bool initialized = initialize();
(void) initialized;
auto io = DB::executeQuery(input, context, true, QueryProcessingStage::Complete);
PullingPipelineExecutor executor(io.pipeline);
Block res;
while (!res && executor.pull(res));
return 0;
}
catch (...)
{
return 1;
}

View File

@ -2222,6 +2222,16 @@ bool ParserMySQLGlobalVariable::parseImpl(Pos & pos, ASTPtr & node, Expected & e
return true;
}
bool ParserExistsExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (ParserKeyword("EXISTS").ignore(pos, expected) && ParserSubquery().parse(pos, node, expected))
{
node = makeASTFunction("exists", node);
return true;
}
return false;
}
bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
@ -2245,6 +2255,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|| ParserFunction().parse(pos, node, expected)
|| ParserQualifiedAsterisk().parse(pos, node, expected)
|| ParserAsterisk().parse(pos, node, expected)
|| ParserExistsExpression().parse(pos, node, expected)
|| ParserCompoundIdentifier(false, true).parse(pos, node, expected)
|| ParserSubstitution().parse(pos, node, expected)
|| ParserMySQLGlobalVariable().parse(pos, node, expected);

View File

@ -325,6 +325,17 @@ protected:
};
/**
* Parse query with EXISTS expression.
*/
class ParserExistsExpression : public IParserBase
{
protected:
const char * getName() const override { return "exists expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** An array or tuple of literals.
* Arrays can also be parsed as an application of [] operator and tuples as an application of 'tuple' function.
* But parsing the whole array/tuple as a whole constant seriously speeds up the analysis of expressions in the case of very large collection.

View File

@ -123,7 +123,7 @@ static inline HTTPRequestHandlerFactoryPtr createInterserverHTTPHandlerFactory(I
addCommonDefaultHandlersFactory(*factory, server);
auto main_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<InterserverIOHTTPHandler>>(server);
main_handler->allowPostAndGetParamsRequest();
main_handler->allowPostGetOrHeadRequest();
factory->addHandler(main_handler);
return factory;
@ -180,7 +180,7 @@ void addDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer
addCommonDefaultHandlersFactory(factory, server);
auto query_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>(server, "query");
query_handler->allowPostAndGetParamsRequest();
query_handler->allowPostGetOrHeadRequest();
factory.addHandler(query_handler);
/// We check that prometheus handler will be served on current (default) port.

View File

@ -103,12 +103,14 @@ public:
});
}
/// Handle POST or GET with params
void allowPostAndGetParamsRequest()
/// Handle Post request or (Get or Head) requests with params
void allowPostGetOrHeadRequest()
{
addFilter([](const auto & request)
{
return request.getURI().find('?') != std::string::npos
return (request.getURI().find('?') != std::string::npos
&& (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD))
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST;
});
}

View File

@ -4,6 +4,8 @@
#include <Storages/StorageDistributed.h>
#include <Disks/StoragePolicy.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/queryToString.h>
@ -88,26 +90,40 @@ static void writeBlockConvert(PushingPipelineExecutor & executor, const Block &
}
static ASTPtr createInsertToRemoteTableQuery(const std::string & database, const std::string & table, const Names & column_names)
{
auto query = std::make_shared<ASTInsertQuery>();
query->table_id = StorageID(database, table);
auto columns = std::make_shared<ASTExpressionList>();
query->columns = columns;
query->children.push_back(columns);
for (const auto & column_name : column_names)
columns->children.push_back(std::make_shared<ASTIdentifier>(column_name));
return query;
}
DistributedSink::DistributedSink(
ContextPtr context_,
StorageDistributed & storage_,
const StorageMetadataPtr & metadata_snapshot_,
const ASTPtr & query_ast_,
const ClusterPtr & cluster_,
bool insert_sync_,
UInt64 insert_timeout_,
StorageID main_table_)
StorageID main_table_,
const Names & columns_to_send_)
: SinkToStorage(metadata_snapshot_->getSampleBlock())
, context(Context::createCopy(context_))
, storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, query_ast(query_ast_)
, query_string(queryToString(query_ast_))
, query_ast(createInsertToRemoteTableQuery(main_table_.database_name, main_table_.table_name, columns_to_send_))
, query_string(queryToString(query_ast))
, cluster(cluster_)
, insert_sync(insert_sync_)
, allow_materialized(context->getSettingsRef().insert_allow_materialized_columns)
, insert_timeout(insert_timeout_)
, main_table(main_table_)
, columns_to_send(columns_to_send_.begin(), columns_to_send_.end())
, log(&Poco::Logger::get("DistributedBlockOutputStream"))
{
const auto & settings = context->getSettingsRef();
@ -128,27 +144,25 @@ void DistributedSink::consume(Chunk chunk)
auto ordinary_block = getHeader().cloneWithColumns(chunk.detachColumns());
if (!allow_materialized)
{
/* They are added by the AddingDefaultBlockOutputStream, and we will get
* different number of columns eventually */
for (const auto & col : metadata_snapshot->getColumns().getMaterialized())
{
if (ordinary_block.has(col.name))
{
ordinary_block.erase(col.name);
LOG_DEBUG(log, "{}: column {} will be removed, because it is MATERIALIZED",
storage.getStorageID().getNameForLogs(), col.name);
}
}
}
if (insert_sync)
writeSync(ordinary_block);
else
writeAsync(ordinary_block);
}
Block DistributedSink::removeSuperfluousColumns(Block block) const
{
for (size_t i = block.columns(); i;)
{
--i;
if (!columns_to_send.contains(block.getByPosition(i).name))
block.erase(i);
}
return block;
}
void DistributedSink::writeAsync(const Block & block)
{
if (random_shard_insert)
@ -403,6 +417,8 @@ void DistributedSink::writeSync(const Block & block)
{
const Settings & settings = context->getSettingsRef();
const auto & shards_info = cluster->getShardsInfo();
Block block_to_send = removeSuperfluousColumns(block);
size_t start = 0;
size_t end = shards_info.size();
@ -415,7 +431,7 @@ void DistributedSink::writeSync(const Block & block)
if (!pool)
{
/// Deferred initialization. Only for sync insertion.
initWritingJobs(block, start, end);
initWritingJobs(block_to_send, start, end);
size_t jobs_count = random_shard_insert ? 1 : (remote_jobs_count + local_jobs_count);
size_t max_threads = std::min<size_t>(settings.max_distributed_connections, jobs_count);
@ -460,7 +476,7 @@ void DistributedSink::writeSync(const Block & block)
finished_jobs_count = 0;
for (size_t shard_index : collections::range(start, end))
for (JobReplica & job : per_shard_jobs[shard_index].replicas_jobs)
pool->scheduleOrThrowOnError(runWritingJob(job, block, num_shards));
pool->scheduleOrThrowOnError(runWritingJob(job, block_to_send, num_shards));
}
catch (...)
{
@ -585,12 +601,13 @@ void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id)
{
const auto & shard_info = cluster->getShardsInfo()[shard_id];
const auto & settings = context->getSettingsRef();
Block block_to_send = removeSuperfluousColumns(block);
if (shard_info.hasInternalReplication())
{
if (shard_info.isLocal() && settings.prefer_localhost_replica)
/// Prefer insert into current instance directly
writeToLocal(block, shard_info.getLocalNodeCount());
writeToLocal(block_to_send, shard_info.getLocalNodeCount());
else
{
const auto & path = shard_info.insertPathForInternalReplication(
@ -598,13 +615,13 @@ void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id)
settings.use_compact_format_in_distributed_parts_names);
if (path.empty())
throw Exception("Directory name for async inserts is empty", ErrorCodes::LOGICAL_ERROR);
writeToShard(block, {path});
writeToShard(block_to_send, {path});
}
}
else
{
if (shard_info.isLocal() && settings.prefer_localhost_replica)
writeToLocal(block, shard_info.getLocalNodeCount());
writeToLocal(block_to_send, shard_info.getLocalNodeCount());
std::vector<std::string> dir_names;
for (const auto & address : cluster->getShardsAddresses()[shard_id])
@ -612,7 +629,7 @@ void DistributedSink::writeAsyncImpl(const Block & block, size_t shard_id)
dir_names.push_back(address.toFullString(settings.use_compact_format_in_distributed_parts_names));
if (!dir_names.empty())
writeToShard(block, dir_names);
writeToShard(block_to_send, dir_names);
}
}

View File

@ -43,11 +43,11 @@ public:
ContextPtr context_,
StorageDistributed & storage_,
const StorageMetadataPtr & metadata_snapshot_,
const ASTPtr & query_ast_,
const ClusterPtr & cluster_,
bool insert_sync_,
UInt64 insert_timeout_,
StorageID main_table_);
StorageID main_table_,
const Names & columns_to_send_);
String getName() const override { return "DistributedSink"; }
void consume(Chunk chunk) override;
@ -65,6 +65,9 @@ private:
void writeAsyncImpl(const Block & block, size_t shard_id = 0);
/// Removes columns which should not be sent to shards.
Block removeSuperfluousColumns(Block block) const;
/// Increments finished_writings_count after each repeat.
void writeToLocal(const Block & block, size_t repeats);
@ -84,7 +87,9 @@ private:
/// Returns the number of blocks was written for each cluster node. Uses during exception handling.
std::string getCurrentStateDescription();
/// Context used for writing to remote tables.
ContextMutablePtr context;
StorageDistributed & storage;
StorageMetadataPtr metadata_snapshot;
ASTPtr query_ast;
@ -102,6 +107,7 @@ private:
/// Sync-related stuff
UInt64 insert_timeout; // in seconds
StorageID main_table;
NameSet columns_to_send;
Stopwatch watch;
Stopwatch watch_current_block;
std::optional<ThreadPool> pool;

View File

@ -159,23 +159,6 @@ ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, co
return modified_query_ast;
}
/// The columns list in the original INSERT query is incorrect because inserted blocks are transformed
/// to the form of the sample block of the Distributed table. So we rewrite it and add all columns from
/// the sample block instead.
ASTPtr createInsertToRemoteTableQuery(const std::string & database, const std::string & table, const Block & sample_block)
{
auto query = std::make_shared<ASTInsertQuery>();
query->table_id = StorageID(database, table);
auto columns = std::make_shared<ASTExpressionList>();
query->columns = columns;
query->children.push_back(columns);
for (const auto & col : sample_block)
columns->children.push_back(std::make_shared<ASTIdentifier>(col.name));
return query;
}
/// Calculate maximum number in file names in directory and all subdirectories.
/// To ensure global order of data blocks yet to be sent across server restarts.
UInt64 getMaximumFileNumber(const std::string & dir_path)
@ -682,17 +665,16 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata
bool insert_sync = settings.insert_distributed_sync || settings.insert_shard_id || owned_cluster;
auto timeout = settings.insert_distributed_timeout;
Block sample_block;
if (!settings.insert_allow_materialized_columns)
sample_block = metadata_snapshot->getSampleBlockNonMaterialized();
Names columns_to_send;
if (settings.insert_allow_materialized_columns)
columns_to_send = metadata_snapshot->getSampleBlock().getNames();
else
sample_block = metadata_snapshot->getSampleBlock();
columns_to_send = metadata_snapshot->getSampleBlockNonMaterialized().getNames();
/// DistributedBlockOutputStream will not own cluster, but will own ConnectionPools of the cluster
return std::make_shared<DistributedSink>(
local_context, *this, metadata_snapshot,
createInsertToRemoteTableQuery(remote_database, remote_table, sample_block),
cluster, insert_sync, timeout, StorageID{remote_database, remote_table});
local_context, *this, metadata_snapshot, cluster, insert_sync, timeout,
StorageID{remote_database, remote_table}, columns_to_send);
}

14
tests/fuzz/README.md Normal file
View File

@ -0,0 +1,14 @@
The list of funtions generated via following query
```
clickhouse-client -q "select concat('\"', name, '\"') from system.functions union all select concat('\"', alias_to, '\"') from system.functions where alias_to != '' " > functions.dict
```
The list of datatypes generated via following query:
```
clickhouse-client -q "select concat('\"', name, '\"') from system.data_type_families union all select concat('\"', alias_to, '\"') from system.data_type_families where alias_to != '' " > datatypes.dict
```
Then merge all dictionaries into one (all.dict)

1466
tests/fuzz/all.dict Normal file

File diff suppressed because it is too large Load Diff

View File

@ -1,2 +1,3 @@
[libfuzzer]
dict = ast.dict
dict = all.dict
jobs = 8

View File

@ -1,38 +0,0 @@
"Nothing"
"UUID"
"Enum8"
"Enum16"
"Enum"
"Date"
"IntervalSecond"
"IntervalMinute"
"IntervalHour"
"IntervalDay"
"IntervalWeek"
"IntervalMonth"
"IntervalQuarter"
"IntervalYear"
"AggregateFunction"
"FixedString"
"LowCardinality"
"DateTime"
"Nullable"
"String"
"UInt8"
"UInt16"
"UInt32"
"UInt64"
"Int8"
"Int16"
"Int32"
"Int64"
"Float32"
"Float64"
"DateTime64"
"Decimal32"
"Decimal64"
"Decimal128"
"Decimal"
"Tuple"
"Nested"
"Array"

View File

@ -0,0 +1,185 @@
"Polygon"
"Ring"
"Point"
"SimpleAggregateFunction"
"MultiPolygon"
"IPv6"
"IntervalSecond"
"IPv4"
"UInt32"
"IntervalYear"
"IntervalQuarter"
"IntervalMonth"
"Int64"
"IntervalDay"
"IntervalHour"
"Int16"
"UInt256"
"LowCardinality"
"AggregateFunction"
"Nothing"
"Decimal256"
"Tuple"
"Array"
"Enum16"
"IntervalMinute"
"FixedString"
"String"
"DateTime"
"Map"
"UUID"
"Decimal64"
"Nullable"
"Enum"
"Int32"
"UInt8"
"Date"
"Decimal32"
"UInt128"
"Float64"
"Nested"
"Int128"
"Decimal128"
"Int8"
"Decimal"
"Int256"
"DateTime64"
"Enum8"
"DateTime32"
"Date32"
"IntervalWeek"
"UInt64"
"UInt16"
"Float32"
"INET6"
"INET4"
"ENUM"
"BINARY"
"NATIONAL CHAR VARYING"
"BINARY VARYING"
"NCHAR LARGE OBJECT"
"NATIONAL CHARACTER VARYING"
"NATIONAL CHARACTER LARGE OBJECT"
"NATIONAL CHARACTER"
"NATIONAL CHAR"
"CHARACTER VARYING"
"LONGBLOB"
"MEDIUMTEXT"
"TEXT"
"TINYBLOB"
"VARCHAR2"
"CHARACTER LARGE OBJECT"
"DOUBLE PRECISION"
"LONGTEXT"
"NVARCHAR"
"INT1 UNSIGNED"
"VARCHAR"
"CHAR VARYING"
"MEDIUMBLOB"
"NCHAR"
"CHAR"
"SMALLINT UNSIGNED"
"TIMESTAMP"
"FIXED"
"TINYTEXT"
"NUMERIC"
"DEC"
"TINYINT UNSIGNED"
"INTEGER UNSIGNED"
"INT UNSIGNED"
"CLOB"
"MEDIUMINT UNSIGNED"
"BOOL"
"SMALLINT"
"INTEGER SIGNED"
"NCHAR VARYING"
"INT SIGNED"
"TINYINT SIGNED"
"BIGINT SIGNED"
"BINARY LARGE OBJECT"
"SMALLINT SIGNED"
"MEDIUMINT"
"INTEGER"
"INT1 SIGNED"
"BIGINT UNSIGNED"
"BYTEA"
"INT"
"SINGLE"
"FLOAT"
"MEDIUMINT SIGNED"
"BOOLEAN"
"DOUBLE"
"INT1"
"CHAR LARGE OBJECT"
"TINYINT"
"BIGINT"
"CHARACTER"
"BYTE"
"BLOB"
"REAL"
"IPv6"
"IPv4"
"Enum"
"FixedString"
"String"
"String"
"String"
"String"
"String"
"String"
"String"
"String"
"String"
"String"
"String"
"String"
"String"
"String"
"Float64"
"String"
"String"
"UInt8"
"String"
"String"
"String"
"String"
"String"
"UInt16"
"DateTime"
"Decimal"
"String"
"Decimal"
"Decimal"
"UInt8"
"UInt32"
"UInt32"
"String"
"UInt32"
"Int8"
"Int16"
"Int32"
"String"
"Int32"
"Int8"
"Int64"
"String"
"Int16"
"Int32"
"Int32"
"Int8"
"UInt64"
"String"
"Int32"
"Float32"
"Float32"
"Int32"
"Int8"
"Float64"
"Int8"
"String"
"Int8"
"Int64"
"String"
"Int8"
"String"
"Float32"

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,183 @@
# Key words (based on keywords from antlr parser)
"ADD"
"AFTER"
"ALIAS"
"ALL"
"ALTER"
"AND"
"ANTI"
"ANY"
"ARRAY"
"AS"
"ASCENDING"
"ASOF"
"AST"
"ASYNC"
"ATTACH"
"BETWEEN"
"BOTH"
"BY"
"CASE"
"CAST"
"CHECK"
"CLEAR"
"CLUSTER"
"CODEC"
"COLLATE"
"COLUMN"
"COMMENT"
"CONSTRAINT"
"CREATE"
"CROSS"
"CUBE"
"DATABASE"
"DATABASES"
"DATE"
"DAY"
"DEDUPLICATE"
"DEFAULT"
"DELAY"
"DELETE"
"DESC"
"DESCENDING"
"DESCRIBE"
"DETACH"
"DICTIONARIES"
"DICTIONARY"
"DISK"
"DISTINCT"
"DISTRIBUTED"
"DROP"
"ELSE"
"END"
"ENGINE"
"EVENTS"
"EXISTS"
"EXPLAIN"
"EXPRESSION"
"EXTRACT"
"FETCHES"
"FINAL"
"FIRST"
"FLUSH"
"FOR"
"FORMAT"
"FREEZE"
"FROM"
"FULL"
"FUNCTION"
"GLOBAL"
"GRANULARITY"
"GROUP"
"HAVING"
"HIERARCHICAL"
"HOUR"
"ID"
"IF"
"ILIKE"
"IN"
"INDEX"
"INF"
"INJECTIVE"
"INNER"
"INSERT"
"INTERVAL"
"INTO"
"IS"
"IS_OBJECT_ID"
"JOIN"
"KEY"
"KILL"
"LAST"
"LAYOUT"
"LEADING"
"LEFT"
"LIFETIME"
"LIKE"
"LIMIT"
"LIVE"
"LOCAL"
"LOGS"
"MATERIALIZE"
"MATERIALIZED"
"MAX"
"MERGES"
"MIN"
"MINUTE"
"MODIFY"
"MONTH"
"MOVE"
"MUTATION"
"NAN_SQL"
"NO"
"NOT"
"NULL_SQL"
"NULLS"
"OFFSET"
"ON"
"OPTIMIZE"
"OR"
"ORDER"
"OUTER"
"OUTFILE"
"PARTITION"
"POPULATE"
"PREWHERE"
"PRIMARY"
"PROJECTION"
"QUARTER"
"RANGE"
"RELOAD"
"REMOVE"
"RENAME"
"REPLACE"
"REPLICA"
"REPLICATED"
"RIGHT"
"ROLLUP"
"SAMPLE"
"SECOND"
"SELECT"
"SEMI"
"SENDS"
"SET"
"SETTINGS"
"SHOW"
"SOURCE"
"START"
"STOP"
"SUBSTRING"
"SYNC"
"SYNTAX"
"SYSTEM"
"TABLE"
"TABLES"
"TEMPORARY"
"TEST"
"THEN"
"TIES"
"TIMEOUT"
"TIMESTAMP"
"TO"
"TOP"
"TOTALS"
"TRAILING"
"TRIM"
"TRUNCATE"
"TTL"
"TYPE"
"UNION"
"UPDATE"
"USE"
"USING"
"UUID"
"VALUES"
"VIEW"
"VOLUME"
"WATCH"
"WEEK"
"WHEN"
"WHERE"
"WITH"
"YEAR"

View File

@ -0,0 +1,3 @@
[libfuzzer]
dict = all.dict
jobs = 8

View File

@ -1,2 +1,3 @@
[libfuzzer]
dict = ast.dict
dict = all.dict
jobs = 8

View File

@ -1,2 +1,3 @@
[libfuzzer]
dict = datatypes.dict
dict = all.dict
jobs = 8

View File

@ -1,2 +1,3 @@
[libfuzzer]
dict = ast.dict
dict = all.dict
jobs = 8

View File

@ -0,0 +1,17 @@
-- { echo }
select exists(select 1);
1
select exists(select 1 except select 1);
0
select exists(select number from numbers(10) where number > 0);
1
select exists(select number from numbers(10) where number < 0);
0
select count() from numbers(10) where exists(select 1 except select 1);
0
select count() from numbers(10) where exists(select 1 intersect select 1);
10
select count() from numbers(10) where exists(select number from numbers(10) where number > 8);
10
select count() from numbers(10) where exists(select number from numbers(10) where number > 9);
0

View File

@ -0,0 +1,11 @@
-- { echo }
select exists(select 1);
select exists(select 1 except select 1);
select exists(select number from numbers(10) where number > 0);
select exists(select number from numbers(10) where number < 0);
select count() from numbers(10) where exists(select 1 except select 1);
select count() from numbers(10) where exists(select 1 intersect select 1);
select count() from numbers(10) where exists(select number from numbers(10) where number > 8);
select count() from numbers(10) where exists(select number from numbers(10) where number > 9);

View File

@ -0,0 +1,18 @@
<yandex>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</yandex>

View File

@ -0,0 +1,116 @@
import pytest
import itertools
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/test_cluster.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', main_configs=['configs/test_cluster.xml'], with_zookeeper=True)
@pytest.fixture(scope="module", autouse=True)
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def cleanup_after_test():
try:
yield
finally:
node1.query("DROP TABLE IF EXISTS dist ON CLUSTER 'test_cluster'")
node1.query("DROP TABLE IF EXISTS local ON CLUSTER 'test_cluster'")
# A default column is used in the sharding key expression.
def test_default_column():
node1.query("CREATE TABLE dist ON CLUSTER 'test_cluster' (x Int32, y Int32 DEFAULT x + 100, z Int32 DEFAULT x + y) ENGINE = Distributed('test_cluster', currentDatabase(), local, y)")
node1.query("CREATE TABLE local ON CLUSTER 'test_cluster' (x Int32, y Int32 DEFAULT x + 200, z Int32 DEFAULT x - y) ENGINE = MergeTree() ORDER BY y")
for insert_sync in [0, 1]:
settings = {'insert_distributed_sync': insert_sync}
# INSERT INTO TABLE dist (x)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query("INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
assert node1.query("SELECT x, y, z FROM local") == TSV([[2, 102, 104], [4, 104, 108]])
assert node2.query("SELECT x, y, z FROM local") == TSV([[1, 101, 102], [3, 103, 106]])
assert node1.query("SELECT x, y, z FROM dist") == TSV([[2, 102, 104], [4, 104, 108], [1, 101, 102], [3, 103, 106]])
# INSERT INTO TABLE dist (x, y)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query("INSERT INTO TABLE dist (x, y) VALUES (1, 11), (2, 22), (3, 33)", settings=settings)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
assert node1.query("SELECT x, y, z FROM local") == TSV([[2, 22, 24]])
assert node2.query("SELECT x, y, z FROM local") == TSV([[1, 11, 12], [3, 33, 36]])
assert node1.query("SELECT x, y, z FROM dist") == TSV([[2, 22, 24], [1, 11, 12], [3, 33, 36]])
# A materialized column is used in the sharding key expression and `insert_allow_materialized_columns` set to 1.
def test_materialized_column_allow_insert_materialized():
node1.query("CREATE TABLE dist ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 100, z Int32 MATERIALIZED x + y) ENGINE = Distributed('test_cluster', currentDatabase(), local, y)")
node1.query("CREATE TABLE local ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 200, z Int32 MATERIALIZED x - y) ENGINE = MergeTree() ORDER BY y")
for insert_sync in [0, 1]:
settings = {'insert_distributed_sync': insert_sync, 'insert_allow_materialized_columns': 1}
# INSERT INTO TABLE dist (x)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query("INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
assert node1.query("SELECT x, y, z FROM local") == TSV([[2, 102, 104], [4, 104, 108]])
assert node2.query("SELECT x, y, z FROM local") == TSV([[1, 101, 102], [3, 103, 106]])
assert node1.query("SELECT x, y, z FROM dist") == TSV([[2, 102, 104], [4, 104, 108], [1, 101, 102], [3, 103, 106]])
# INSERT INTO TABLE dist (x, y)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query("INSERT INTO TABLE dist (x, y) VALUES (1, 11), (2, 22), (3, 33)", settings=settings)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
assert node1.query("SELECT x, y, z FROM local") == TSV([[2, 22, 24]])
assert node2.query("SELECT x, y, z FROM local") == TSV([[1, 11, 12], [3, 33, 36]])
assert node1.query("SELECT x, y, z FROM dist") == TSV([[2, 22, 24], [1, 11, 12], [3, 33, 36]])
# A materialized column is used in the sharding key expression and `insert_allow_materialized_columns` set to 0.
def test_materialized_column_disallow_insert_materialized():
node1.query("CREATE TABLE dist ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 100, z Int32 MATERIALIZED x + y) ENGINE = Distributed('test_cluster', currentDatabase(), local, y)")
node1.query("CREATE TABLE local ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 200, z Int32 MATERIALIZED x - y) ENGINE = MergeTree() ORDER BY y")
for insert_sync in [0, 1]:
settings = {'insert_distributed_sync': insert_sync, 'insert_allow_materialized_columns': 0}
# INSERT INTO TABLE dist (x)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query("INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
assert node1.query("SELECT x, y, z FROM local") == TSV([[2, 202, -200], [4, 204, -200]])
assert node2.query("SELECT x, y, z FROM local") == TSV([[1, 201, -200], [3, 203, -200]])
assert node1.query("SELECT x, y, z FROM dist") == TSV([[2, 202, -200], [4, 204, -200], [1, 201, -200], [3, 203, -200]])
# INSERT INTO TABLE dist (x, y)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
expected_error = "Cannot insert column y, because it is MATERIALIZED column"
assert expected_error in node1.query_and_get_error("INSERT INTO TABLE dist (x, y) VALUES (1, 11), (2, 22), (3, 33)", settings=settings)
# Almost the same as the previous test `test_materialized_column_disallow_insert_materialized`, but the sharding key has different values.
def test_materialized_column_disallow_insert_materialized_different_shards():
node1.query("CREATE TABLE dist ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 101, z Int32 MATERIALIZED x + y) ENGINE = Distributed('test_cluster', currentDatabase(), local, y)")
node1.query("CREATE TABLE local ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 200, z Int32 MATERIALIZED x - y) ENGINE = MergeTree() ORDER BY y")
for insert_sync in [0, 1]:
settings = {'insert_distributed_sync': insert_sync, 'insert_allow_materialized_columns': 0}
# INSERT INTO TABLE dist (x)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query("INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
assert node1.query("SELECT x, y, z FROM local") == TSV([[1, 201, -200], [3, 203, -200]])
assert node2.query("SELECT x, y, z FROM local") == TSV([[2, 202, -200], [4, 204, -200]])
assert node1.query("SELECT x, y, z FROM dist") == TSV([[1, 201, -200], [3, 203, -200], [2, 202, -200], [4, 204, -200]])

View File

@ -0,0 +1,3 @@
< HTTP/1.1 200 OK
< HTTP/1.1 501 Not Implemented
< HTTP/1.1 200 OK

View File

@ -0,0 +1,9 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CURL "${CLICKHOUSE_URL}" -X GET -d "SELECT 1" -vs 2>&1 | grep "OK"
$CLICKHOUSE_CURL "${CLICKHOUSE_URL}" -X aaa -vs 2>&1 | grep "Not Implemented"
$CLICKHOUSE_CURL "${CLICKHOUSE_URL}" -d "SELECT 1" -vs 2>&1 | grep "OK"

View File

@ -0,0 +1,19 @@
-- { echoOn }
SELECT JSONExtractInt('[1]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractUInt('[1]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractBool('[1]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractFloat('[1]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractString('["a"]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractArrayRaw('["1"]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractKeysAndValuesRaw('["1"]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractKeysAndValues('["1"]', toNullable(1)); -- { serverError ILLEGAL_COLUMN }
SELECT JSONExtract('[1]', toNullable(1), 'Nullable(Int)');
1
SELECT JSONExtract('[1]', toNullable(1), 'Nullable(UInt8)');
1
SELECT JSONExtract('[1]', toNullable(1), 'Nullable(Bool)');
1
SELECT JSONExtract('[1]', toNullable(1), 'Nullable(Float)');
1
SELECT JSONExtract('["a"]', toNullable(1), 'Nullable(String)');
a

View File

@ -0,0 +1,21 @@
-- Tags: no-fasttest
-- to avoid merging Tags and echoOn
SELECT 1 FORMAT Null;
-- { echoOn }
SELECT JSONExtractInt('[1]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractUInt('[1]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractBool('[1]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractFloat('[1]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractString('["a"]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractArrayRaw('["1"]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractKeysAndValuesRaw('["1"]', toNullable(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT JSONExtractKeysAndValues('["1"]', toNullable(1)); -- { serverError ILLEGAL_COLUMN }
SELECT JSONExtract('[1]', toNullable(1), 'Nullable(Int)');
SELECT JSONExtract('[1]', toNullable(1), 'Nullable(UInt8)');
SELECT JSONExtract('[1]', toNullable(1), 'Nullable(Bool)');
SELECT JSONExtract('[1]', toNullable(1), 'Nullable(Float)');
SELECT JSONExtract('["a"]', toNullable(1), 'Nullable(String)');