Merge branch 'master' into update-minio

This commit is contained in:
mergify[bot] 2021-10-06 05:45:18 +00:00 committed by GitHub
commit 6eec0d76a7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
55 changed files with 3404 additions and 332 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

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

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

@ -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

@ -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.

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);