mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Moved table functions to separate library; fixed errors; improved testability [#CLICKHOUSE-31].
This commit is contained in:
parent
5548f1fdd9
commit
45c76803a9
@ -42,7 +42,6 @@ set(dbms_headers)
|
||||
set(dbms_sources)
|
||||
|
||||
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
|
||||
add_headers_and_sources(dbms src/TableFunctions)
|
||||
add_headers_and_sources(dbms src/Parsers)
|
||||
add_headers_and_sources(dbms src/Analyzers)
|
||||
add_headers_and_sources(dbms src/Core)
|
||||
@ -70,9 +69,8 @@ list (APPEND dbms_headers ${CONFIG_VERSION} ${CONFIG_COMMON})
|
||||
|
||||
list (APPEND dbms_sources src/Functions/IFunction.cpp src/Functions/FunctionFactory.cpp src/Functions/DataTypeTraits.cpp)
|
||||
list (APPEND dbms_headers src/Functions/IFunction.h src/Functions/FunctionFactory.h src/Functions/DataTypeTraits.h)
|
||||
|
||||
list (APPEND dbms_sources
|
||||
src/AggregateFunctions/AggregateFunctionFactory.cpp
|
||||
src/AggregateFunctions/AggregateFunctionState.cpp
|
||||
src/AggregateFunctions/AggregateFunctionFactory.cpp
|
||||
src/AggregateFunctions/AggregateFunctionState.cpp
|
||||
src/AggregateFunctions/AggregateFunctionArray.cpp
|
||||
@ -80,22 +78,21 @@ list (APPEND dbms_sources
|
||||
src/AggregateFunctions/AggregateFunctionForEach.cpp
|
||||
src/AggregateFunctions/AggregateFunctionIf.cpp
|
||||
src/AggregateFunctions/AggregateFunctionMerge.cpp
|
||||
src/AggregateFunctions/AggregateFunctionCount.cpp
|
||||
)
|
||||
src/AggregateFunctions/AggregateFunctionCount.cpp)
|
||||
|
||||
list (APPEND dbms_headers
|
||||
src/AggregateFunctions/IAggregateFunction.h
|
||||
src/AggregateFunctions/AggregateFunctionFactory.h
|
||||
src/AggregateFunctions/AggregateFunctionState.h
|
||||
src/AggregateFunctions/AggregateFunctionFactory.h
|
||||
src/AggregateFunctions/AggregateFunctionState.h
|
||||
src/AggregateFunctions/AggregateFunctionArray.h
|
||||
src/AggregateFunctions/AggregateFunctionNull.h
|
||||
src/AggregateFunctions/AggregateFunctionForEach.h
|
||||
src/AggregateFunctions/AggregateFunctionIf.h
|
||||
src/AggregateFunctions/AggregateFunctionMerge.h
|
||||
src/AggregateFunctions/AggregateFunctionCount.h
|
||||
)
|
||||
src/AggregateFunctions/AggregateFunctionCount.h)
|
||||
|
||||
list (APPEND dbms_sources src/TableFunctions/TableFunctionFactory.cpp)
|
||||
list (APPEND dbms_headers src/TableFunctions/ITableFunction.h src/TableFunctions/TableFunctionFactory.h)
|
||||
|
||||
|
||||
list(REMOVE_ITEM dbms_sources
|
||||
|
@ -52,12 +52,12 @@ void AggregateFunctionFactory::registerFunction(const String & name, Creator cre
|
||||
" a null constructor", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!aggregate_functions.emplace(name, creator).second)
|
||||
throw Exception("AggregateFunctionFactory: the aggregate function name " + name + " is not unique",
|
||||
throw Exception("AggregateFunctionFactory: the aggregate function name '" + name + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (case_sensitiveness == CaseInsensitive
|
||||
&& !case_insensitive_aggregate_functions.emplace(Poco::toLower(name), creator).second)
|
||||
throw Exception("AggregateFunctionFactory: the case insensitive aggregate function name " + name + " is not unique",
|
||||
throw Exception("AggregateFunctionFactory: the case insensitive aggregate function name '" + name + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Analyzers/ExecuteTableFunctions.h>
|
||||
#include <Analyzers/AnalyzeResultOfQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
@ -32,7 +32,7 @@ static void processTableFunction(const ASTPtr & ast_table_function, const Contex
|
||||
return;
|
||||
|
||||
/// Obtain table function
|
||||
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(function.name, context);
|
||||
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(function.name, context);
|
||||
/// Execute it and store result
|
||||
StoragePtr table = table_function_ptr->execute(ast_table_function, context);
|
||||
result_map[ast_hash] = table;
|
||||
|
@ -8,7 +8,8 @@ add_executable(analyze_columns analyze_columns.cpp)
|
||||
target_link_libraries(analyze_columns dbms clickhouse_storages_system)
|
||||
|
||||
add_executable(type_and_constant_inference type_and_constant_inference.cpp)
|
||||
target_link_libraries(type_and_constant_inference clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions dbms)
|
||||
target_link_libraries(type_and_constant_inference
|
||||
clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions dbms)
|
||||
|
||||
add_executable(analyze_result_of_query analyze_result_of_query.cpp)
|
||||
target_link_libraries(analyze_result_of_query dbms clickhouse_storages_system)
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
|
||||
|
||||
/// Parses query from stdin and print data types of expressions; and for constant expressions, print its values.
|
||||
@ -28,6 +29,7 @@ try
|
||||
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
|
||||
ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
||||
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
|
||||
|
@ -191,6 +191,7 @@ private:
|
||||
APPLY_FOR_LIMITS(EXTRACT_LIMIT)
|
||||
#undef EXTRACT_LIMIT
|
||||
|
||||
/// FIXME Why do we need this?
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
}
|
||||
|
@ -70,11 +70,11 @@ template<> struct MinCounterTypeHelper<3> { using Type = UInt64; };
|
||||
/// Used in HyperLogLogCounter in order to spend memory efficiently.
|
||||
template<UInt64 MaxValue> struct MinCounterType
|
||||
{
|
||||
typedef typename MinCounterTypeHelper<
|
||||
using Type = typename MinCounterTypeHelper<
|
||||
(MaxValue >= 1 << 8) +
|
||||
(MaxValue >= 1 << 16) +
|
||||
(MaxValue >= 1ULL << 32)
|
||||
>::Type Type;
|
||||
>::Type;
|
||||
};
|
||||
|
||||
/// Denominator of expression for HyperLogLog algorithm.
|
||||
|
@ -58,9 +58,9 @@ int main(int argc, char ** argv)
|
||||
}
|
||||
|
||||
{
|
||||
typedef HashSet<
|
||||
using Cont = HashSet<
|
||||
DB::UInt128,
|
||||
DB::UInt128TrivialHash> Cont;
|
||||
DB::UInt128TrivialHash>;
|
||||
Cont cont;
|
||||
|
||||
std::string dump;
|
||||
|
@ -53,14 +53,14 @@ struct __attribute__((__aligned__(64))) AlignedSmallLock : public SmallLock
|
||||
using Mutex = std::mutex;
|
||||
|
||||
|
||||
/*typedef HashTableWithSmallLocks<
|
||||
/*using MapSmallLocks = HashTableWithSmallLocks<
|
||||
Key,
|
||||
HashTableCellWithLock<
|
||||
Key,
|
||||
HashMapCell<Key, Value, DefaultHash<Key> > >,
|
||||
DefaultHash<Key>,
|
||||
HashTableGrower<21>,
|
||||
HashTableAllocator> MapSmallLocks;*/
|
||||
HashTableAllocator>;*/
|
||||
|
||||
|
||||
void aggregate1(Map & map, Source::const_iterator begin, Source::const_iterator end)
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <memory>
|
||||
#include <unordered_map>
|
||||
#include <common/singleton.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -13,6 +14,11 @@ class Context;
|
||||
class IFunction;
|
||||
using FunctionPtr = std::shared_ptr<IFunction>;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
/** Creates function by name.
|
||||
* Function could use for initialization (take ownership of shared_ptr, for example)
|
||||
@ -23,7 +29,7 @@ class FunctionFactory : public Singleton<FunctionFactory>
|
||||
friend class StorageSystemFunctions;
|
||||
|
||||
private:
|
||||
typedef FunctionPtr (*Creator)(const Context & context); /// Not std::function, for lower object size and less indirection.
|
||||
using Creator = FunctionPtr(*)(const Context & context); /// Not std::function, for lower object size and less indirection.
|
||||
std::unordered_map<std::string, Creator> functions;
|
||||
|
||||
public:
|
||||
@ -33,10 +39,13 @@ public:
|
||||
FunctionPtr tryGet(const std::string & name, const Context & context) const; /// Returns nullptr if not found.
|
||||
|
||||
/// No locking, you must register all functions before usage of get, tryGet.
|
||||
template <typename F> void registerFunction()
|
||||
template <typename Function> void registerFunction()
|
||||
{
|
||||
static_assert(std::is_same<decltype(&F::create), Creator>::value, "F::create has incorrect type");
|
||||
functions[F::name] = &F::create;
|
||||
static_assert(std::is_same<decltype(&Function::create), Creator>::value, "Function::create has incorrect type");
|
||||
|
||||
if (!functions.emplace(Function::name, &Function::create).second)
|
||||
throw Exception("FunctionFactory: the function name '" + std::string(Function::name) + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -1398,8 +1398,8 @@ bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn *
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
const typename ColumnVector<T>::Container_t & values = nested->getData();
|
||||
|
||||
typedef ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(T)> > Set;
|
||||
using Set = ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
@ -1444,8 +1444,8 @@ bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn *
|
||||
return false;
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
|
||||
typedef ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(StringRef)> > Set;
|
||||
using Set = ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
@ -1511,8 +1511,8 @@ bool FunctionArrayUniq::execute128bit(
|
||||
if (keys_bytes > 16)
|
||||
return false;
|
||||
|
||||
typedef ClearableHashSet<UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)> > Set;
|
||||
using Set = ClearableHashSet<UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
/// Suppose that, for a given row, each of the N columns has an array whose length is M.
|
||||
/// Denote arr_i each of these arrays (1 <= i <= N). Then the following is performed:
|
||||
@ -1572,8 +1572,8 @@ void FunctionArrayUniq::executeHashed(
|
||||
{
|
||||
size_t count = columns.size();
|
||||
|
||||
typedef ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)> > Set;
|
||||
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
Set set;
|
||||
size_t prev_off = 0;
|
||||
@ -1724,8 +1724,8 @@ bool FunctionArrayEnumerateUniq::executeNumber(const ColumnArray * array, const
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
const typename ColumnVector<T>::Container_t & values = nested->getData();
|
||||
|
||||
typedef ClearableHashMap<T, UInt32, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(T)> > ValuesToIndices;
|
||||
using ValuesToIndices = ClearableHashMap<T, UInt32, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
@ -1769,8 +1769,8 @@ bool FunctionArrayEnumerateUniq::executeString(const ColumnArray * array, const
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
|
||||
size_t prev_off = 0;
|
||||
typedef ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(StringRef)> > ValuesToIndices;
|
||||
using ValuesToIndices = ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
@ -1838,8 +1838,8 @@ bool FunctionArrayEnumerateUniq::execute128bit(
|
||||
if (keys_bytes > 16)
|
||||
return false;
|
||||
|
||||
typedef ClearableHashMap<UInt128, UInt32, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)> > ValuesToIndices;
|
||||
using ValuesToIndices = ClearableHashMap<UInt128, UInt32, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
ValuesToIndices indices;
|
||||
size_t prev_off = 0;
|
||||
@ -1884,8 +1884,8 @@ void FunctionArrayEnumerateUniq::executeHashed(
|
||||
{
|
||||
size_t count = columns.size();
|
||||
|
||||
typedef ClearableHashMap<UInt128, UInt32, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)> > ValuesToIndices;
|
||||
using ValuesToIndices = ClearableHashMap<UInt128, UInt32, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
ValuesToIndices indices;
|
||||
size_t prev_off = 0;
|
||||
|
@ -1344,7 +1344,7 @@ public:
|
||||
class FunctionArrayReverse : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "reverse";
|
||||
static constexpr auto name = "arrayReverse";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
@ -193,8 +193,7 @@ struct NameUpperUTF8
|
||||
};
|
||||
|
||||
|
||||
typedef FunctionStringToString<LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>, NameLowerUTF8>
|
||||
FunctionLowerUTF8;
|
||||
typedef FunctionStringToString<LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>, NameUpperUTF8>
|
||||
FunctionUpperUTF8;
|
||||
using FunctionLowerUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>, NameLowerUTF8>;
|
||||
using FunctionUpperUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>, NameUpperUTF8>;
|
||||
|
||||
}
|
||||
|
@ -25,20 +25,20 @@ namespace DB
|
||||
namespace NumberTraits
|
||||
{
|
||||
|
||||
using Unsigned = boost::mpl::false_ ;
|
||||
using Unsigned = boost::mpl::false_;
|
||||
using Signed = boost::mpl::true_ ;
|
||||
|
||||
using Integer = boost::mpl::false_ ;
|
||||
using Floating = boost::mpl::true_ ;
|
||||
using Integer = boost::mpl::false_;
|
||||
using Floating = boost::mpl::true_;
|
||||
|
||||
using HasNull = boost::mpl::true_;
|
||||
using HasNoNull = boost::mpl::false_;
|
||||
|
||||
using Bits0 = boost::mpl::int_<0> ;
|
||||
using Bits8 = boost::mpl::int_<8> ;
|
||||
using Bits16 = boost::mpl::int_<16> ;
|
||||
using Bits32 = boost::mpl::int_<32> ;
|
||||
using Bits64 = boost::mpl::int_<64> ;
|
||||
using Bits0 = boost::mpl::int_<0>;
|
||||
using Bits8 = boost::mpl::int_<8>;
|
||||
using Bits16 = boost::mpl::int_<16>;
|
||||
using Bits32 = boost::mpl::int_<32>;
|
||||
using Bits64 = boost::mpl::int_<64>;
|
||||
using BitsTooMany = boost::mpl::int_<1024>;
|
||||
|
||||
struct Error {};
|
||||
@ -80,18 +80,18 @@ struct Traits<Nullable<T>>
|
||||
using Nullity = HasNull;
|
||||
};
|
||||
|
||||
template <> struct Traits<void> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits0 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<void> { using Sign = Unsigned; using Floatness = Integer; using Bits = Bits0; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Null> : Traits<Nullable<void>> {};
|
||||
template <> struct Traits<UInt8> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits8 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<UInt16> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits16 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<UInt32> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits32 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<UInt64> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits64 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Int8> { typedef Signed Sign; typedef Integer Floatness; typedef Bits8 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Int16> { typedef Signed Sign; typedef Integer Floatness; typedef Bits16 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Int32> { typedef Signed Sign; typedef Integer Floatness; typedef Bits32 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Int64> { typedef Signed Sign; typedef Integer Floatness; typedef Bits64 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Float32> { typedef Signed Sign; typedef Floating Floatness; typedef Bits32 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<Float64> { typedef Signed Sign; typedef Floating Floatness; typedef Bits64 Bits; typedef HasNoNull Nullity; };
|
||||
template <> struct Traits<UInt8> { using Sign = Unsigned; using Floatness = Integer; using Bits = Bits8; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<UInt16> { using Sign = Unsigned; using Floatness = Integer; using Bits = Bits16; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<UInt32> { using Sign = Unsigned; using Floatness = Integer; using Bits = Bits32; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<UInt64> { using Sign = Unsigned; using Floatness = Integer; using Bits = Bits64; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Int8> { using Sign = Signed; using Floatness = Integer; using Bits = Bits8; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Int16> { using Sign = Signed; using Floatness = Integer; using Bits = Bits16; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Int32> { using Sign = Signed; using Floatness = Integer; using Bits = Bits32; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Int64> { using Sign = Signed; using Floatness = Integer; using Bits = Bits64; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Float32> { using Sign = Signed; using Floatness = Floating; using Bits = Bits32; using Nullity = HasNoNull; };
|
||||
template <> struct Traits<Float64> { using Sign = Signed; using Floatness = Floating; using Bits = Bits64; using Nullity = HasNoNull; };
|
||||
|
||||
template <typename Sign, typename Floatness, typename Bits, typename Nullity> struct Construct;
|
||||
|
||||
@ -101,10 +101,10 @@ struct Construct<Sign, Floatness, Bits, HasNull>
|
||||
using Type = Nullable<typename Construct<Sign, Floatness, Bits, HasNoNull>::Type>;
|
||||
};
|
||||
|
||||
template <> struct Construct<Unsigned, Integer, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Signed, Integer, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Signed, Floating, Bits0, HasNull> { using Type = Null; };
|
||||
template <> struct Construct<Signed, Floating, Bits0, HasNull> { using Type = Null; };
|
||||
|
||||
template <typename Sign, typename Floatness>
|
||||
struct Construct<Sign, Floatness, BitsTooMany, HasNull>
|
||||
@ -118,26 +118,26 @@ struct Construct<Sign, Floatness, BitsTooMany, HasNoNull>
|
||||
using Type = Error;
|
||||
};
|
||||
|
||||
template <> struct Construct<Unsigned, Integer, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Signed, Integer, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Signed, Integer, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Signed, Floating, Bits0, HasNoNull> { using Type = void; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits8, HasNoNull> { using Type = UInt8 ; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits16, HasNoNull> { using Type = UInt16 ; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits32, HasNoNull> { using Type = UInt32 ; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits64, HasNoNull> { using Type = UInt64 ; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits8, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits16, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits32, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits64, HasNoNull> { using Type = Float64 ; };
|
||||
template <> struct Construct<Signed, Integer, Bits8, HasNoNull> { using Type = Int8 ; };
|
||||
template <> struct Construct<Signed, Integer, Bits16, HasNoNull> { using Type = Int16 ; };
|
||||
template <> struct Construct<Signed, Integer, Bits32, HasNoNull> { using Type = Int32 ; };
|
||||
template <> struct Construct<Signed, Integer, Bits64, HasNoNull> { using Type = Int64 ; };
|
||||
template <> struct Construct<Signed, Floating, Bits8, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Signed, Floating, Bits16, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Signed, Floating, Bits32, HasNoNull> { using Type = Float32 ; };
|
||||
template <> struct Construct<Signed, Floating, Bits64, HasNoNull> { using Type = Float64 ; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits8, HasNoNull> { using Type = UInt8; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits16, HasNoNull> { using Type = UInt16; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits32, HasNoNull> { using Type = UInt32; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits64, HasNoNull> { using Type = UInt64; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits8, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits16, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits32, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits64, HasNoNull> { using Type = Float64; };
|
||||
template <> struct Construct<Signed, Integer, Bits8, HasNoNull> { using Type = Int8; };
|
||||
template <> struct Construct<Signed, Integer, Bits16, HasNoNull> { using Type = Int16; };
|
||||
template <> struct Construct<Signed, Integer, Bits32, HasNoNull> { using Type = Int32; };
|
||||
template <> struct Construct<Signed, Integer, Bits64, HasNoNull> { using Type = Int64; };
|
||||
template <> struct Construct<Signed, Floating, Bits8, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Signed, Floating, Bits16, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Signed, Floating, Bits32, HasNoNull> { using Type = Float32; };
|
||||
template <> struct Construct<Signed, Floating, Bits64, HasNoNull> { using Type = Float64; };
|
||||
|
||||
template <typename T>
|
||||
inline bool isErrorType()
|
||||
@ -170,20 +170,20 @@ struct UpdateNullity
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfAdditionMultiplication
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
|
||||
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type;
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct ResultOfSubtraction
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
Signed,
|
||||
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
|
||||
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type;
|
||||
};
|
||||
|
||||
/** When dividing, you always get a floating-point number.
|
||||
@ -197,50 +197,50 @@ template <typename A, typename B> struct ResultOfFloatingPointDivision
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfIntegerDivision
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
Integer,
|
||||
typename Traits<A>::Bits,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type;
|
||||
};
|
||||
|
||||
/** Division with remainder you get a number with the same number of bits as in divisor.
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfModulo
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
Integer,
|
||||
typename Traits<B>::Bits,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type;
|
||||
};
|
||||
|
||||
template <typename A> struct ResultOfNegate
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
Signed,
|
||||
typename Traits<A>::Floatness,
|
||||
typename boost::mpl::if_<
|
||||
typename Traits<A>::Sign,
|
||||
typename Traits<A>::Bits,
|
||||
typename Next<typename Traits<A>::Bits>::Type>::type,
|
||||
typename Traits<A>::Nullity>::Type Type;
|
||||
typename Traits<A>::Nullity>::Type;
|
||||
};
|
||||
|
||||
template <typename A> struct ResultOfAbs
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
Unsigned,
|
||||
typename Traits<A>::Floatness,
|
||||
typename Traits <A>::Bits,
|
||||
typename Traits<A>::Nullity>::Type Type;
|
||||
typename Traits<A>::Nullity>::Type;
|
||||
};
|
||||
|
||||
/** For bitwise operations, an integer is obtained with number of bits is equal to the maximum of the arguments.
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfBit
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
Integer,
|
||||
typename boost::mpl::max<
|
||||
@ -252,16 +252,16 @@ template <typename A, typename B> struct ResultOfBit
|
||||
typename Traits<B>::Floatness,
|
||||
Bits64,
|
||||
typename Traits<B>::Bits>::type>::type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type;
|
||||
};
|
||||
|
||||
template <typename A> struct ResultOfBitNot
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename Traits<A>::Sign,
|
||||
Integer,
|
||||
typename Traits<A>::Bits,
|
||||
typename Traits<A>::Nullity>::Type Type;
|
||||
typename Traits<A>::Nullity>::Type;
|
||||
};
|
||||
|
||||
|
||||
@ -278,7 +278,7 @@ template <typename A> struct ResultOfBitNot
|
||||
template <typename A, typename B>
|
||||
struct ResultOfIf
|
||||
{
|
||||
typedef
|
||||
using Type =
|
||||
/// 1)
|
||||
typename boost::mpl::if_<
|
||||
typename boost::mpl::equal_to<typename Traits<A>::Bits, Bits0>::type,
|
||||
@ -331,13 +331,13 @@ struct ResultOfIf
|
||||
typename Traits<B>::Bits,
|
||||
typename ExactNext<typename Traits<B>::Bits>::Type>::type>::type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type
|
||||
>::Type>::type>::type>::type>::type Type;
|
||||
>::Type>::type>::type>::type>::type;
|
||||
};
|
||||
|
||||
/** Before applying operator `%` and bitwise operations, operands are casted to whole numbers. */
|
||||
template <typename A> struct ToInteger
|
||||
{
|
||||
typedef typename Construct<
|
||||
using Type = typename Construct<
|
||||
typename Traits<A>::Sign,
|
||||
Integer,
|
||||
typename boost::mpl::if_<
|
||||
@ -345,7 +345,7 @@ template <typename A> struct ToInteger
|
||||
Bits64,
|
||||
typename Traits<A>::Bits>::type,
|
||||
typename Traits<A>::Nullity
|
||||
>::Type Type;
|
||||
>::Type;
|
||||
};
|
||||
|
||||
|
||||
@ -353,18 +353,18 @@ template <typename A> struct ToInteger
|
||||
// NOTE: This case is applied for 64-bit integers only (for backward compability), but colud be used for any-bit integers
|
||||
template <typename A, typename B>
|
||||
using LeastGreatestSpecialCase = std::integral_constant<bool, std::is_integral<A>::value && std::is_integral<B>::value
|
||||
&& (8 == sizeof(A) && sizeof(A) == sizeof(B))
|
||||
&& (std::is_signed<A>::value ^ std::is_signed<B>::value)>;
|
||||
&& (8 == sizeof(A) && sizeof(A) == sizeof(B))
|
||||
&& (std::is_signed<A>::value ^ std::is_signed<B>::value)>;
|
||||
|
||||
template <typename A, typename B>
|
||||
using ResultOfLeast = std::conditional_t<LeastGreatestSpecialCase<A, B>::value,
|
||||
typename Construct<Signed, Integer, typename Traits<A>::Bits, HasNoNull>::Type,
|
||||
typename ResultOfIf<A, B>::Type>;
|
||||
typename Construct<Signed, Integer, typename Traits<A>::Bits, HasNoNull>::Type,
|
||||
typename ResultOfIf<A, B>::Type>;
|
||||
|
||||
template <typename A, typename B>
|
||||
using ResultOfGreatest = std::conditional_t<LeastGreatestSpecialCase<A, B>::value,
|
||||
typename Construct<Unsigned, Integer, typename Traits<A>::Bits, HasNoNull>::Type,
|
||||
typename ResultOfIf<A, B>::Type>;
|
||||
typename Construct<Unsigned, Integer, typename Traits<A>::Bits, HasNoNull>::Type,
|
||||
typename ResultOfIf<A, B>::Type>;
|
||||
|
||||
/// Notes on type composition.
|
||||
///
|
||||
@ -542,7 +542,7 @@ namespace
|
||||
template <typename T1, typename T2>
|
||||
constexpr bool areSimilarTypes()
|
||||
{
|
||||
return std::is_same<
|
||||
return std::is_same<
|
||||
typename std::tuple_element<0, T1>::type,
|
||||
typename std::tuple_element<0, T2>::type
|
||||
>::value &&
|
||||
@ -624,17 +624,17 @@ public:
|
||||
/// Compute the product of two enriched numeric types.
|
||||
/// Case when a source type or the resulting type does not map to any ordinary type.
|
||||
|
||||
#define DEFINE_TYPE_PRODUCT_RULE(T1, T2, T3) \
|
||||
template <typename A, typename B> \
|
||||
struct TypeProduct< \
|
||||
A, \
|
||||
B, \
|
||||
typename std::enable_if< \
|
||||
!isOrdinaryPair<A, B>() && \
|
||||
areSimilarPairs<A, B, T1, T2>() \
|
||||
>::type> \
|
||||
{ \
|
||||
using Type = typename T3<typename CombinedNullity<A, B>::Type>; \
|
||||
#define DEFINE_TYPE_PRODUCT_RULE(T1, T2, T3) \
|
||||
template <typename A, typename B> \
|
||||
struct TypeProduct< \
|
||||
A, \
|
||||
B, \
|
||||
typename std::enable_if< \
|
||||
!isOrdinaryPair<A, B>() && \
|
||||
areSimilarPairs<A, B, T1, T2>() \
|
||||
>::type> \
|
||||
{ \
|
||||
using Type = typename T3<typename CombinedNullity<A, B>::Type>;\
|
||||
}
|
||||
|
||||
DEFINE_TYPE_PRODUCT_RULE(Enriched::Int8, Enriched::UInt16, Enriched::IntFloat32);
|
||||
|
@ -16,8 +16,6 @@
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <DataStreams/FormatFactory.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MarkCache.h>
|
||||
#include <Storages/MergeTree/BackgroundProcessingPool.h>
|
||||
@ -82,8 +80,6 @@ namespace ErrorCodes
|
||||
extern const int SESSION_IS_LOCKED;
|
||||
}
|
||||
|
||||
class TableFunctionFactory;
|
||||
|
||||
|
||||
/** Set of known objects (environment), that could be used in query.
|
||||
* Shared (global) part. Order of members (especially, order of destruction) is very important.
|
||||
@ -109,7 +105,6 @@ struct ContextShared
|
||||
String tmp_path; /// The path to the temporary files that occur when processing the request.
|
||||
String flags_path; /// Path to the directory with some control flags for server maintenance.
|
||||
Databases databases; /// List of databases and tables in them.
|
||||
TableFunctionFactory table_function_factory; /// Table functions.
|
||||
FormatFactory format_factory; /// Formats.
|
||||
mutable std::shared_ptr<EmbeddedDictionaries> embedded_dictionaries; /// Metrica's dictionaeis. Have lazy initialization.
|
||||
mutable std::shared_ptr<ExternalDictionaries> external_dictionaries;
|
||||
@ -244,7 +239,6 @@ Context::~Context()
|
||||
}
|
||||
|
||||
|
||||
const TableFunctionFactory & Context::getTableFunctionFactory() const { return shared->table_function_factory; }
|
||||
InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; }
|
||||
|
||||
std::unique_lock<Poco::Mutex> Context::getLock() const
|
||||
|
@ -33,8 +33,6 @@ namespace DB
|
||||
|
||||
struct ContextShared;
|
||||
class QuotaForIntervals;
|
||||
class TableFunctionFactory;
|
||||
class AggregateFunctionFactory;
|
||||
class EmbeddedDictionaries;
|
||||
class ExternalDictionaries;
|
||||
class InterserverIOHandler;
|
||||
@ -196,7 +194,6 @@ public:
|
||||
/// Set a setting by name. Read the value in text form from a string (for example, from a config, or from a URL parameter).
|
||||
void setSetting(const String & name, const std::string & value);
|
||||
|
||||
const TableFunctionFactory & getTableFunctionFactory() const;
|
||||
const EmbeddedDictionaries & getEmbeddedDictionaries() const;
|
||||
const ExternalDictionaries & getExternalDictionaries() const;
|
||||
void tryCreateEmbeddedDictionaries() const;
|
||||
|
@ -139,7 +139,7 @@ void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_)
|
||||
if (query_table && typeid_cast<const ASTFunction *>(query_table.get()))
|
||||
{
|
||||
/// Get the table function
|
||||
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(typeid_cast<const ASTFunction *>(query_table.get())->name, context);
|
||||
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(typeid_cast<const ASTFunction *>(query_table.get())->name, context);
|
||||
/// Run it and remember the result
|
||||
storage = table_function_ptr->execute(query_table, context);
|
||||
}
|
||||
|
@ -61,12 +61,12 @@ struct Grower : public HashTableGrower<2>
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
typedef HashMapWithDump<
|
||||
using Map = HashMapWithDump<
|
||||
StringRef,
|
||||
UInt64,
|
||||
SimpleHash,
|
||||
Grower,
|
||||
HashTableAllocatorWithStackMemory<4 * 24> > Map;
|
||||
HashTableAllocatorWithStackMemory<4 * 24>>;
|
||||
|
||||
Map map;
|
||||
|
||||
|
@ -8,10 +8,10 @@ add_library(clickhouse-server
|
||||
StatusFile.cpp
|
||||
ReplicasStatusHandler.cpp
|
||||
)
|
||||
target_link_libraries(clickhouse-server daemon clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions)
|
||||
target_link_libraries(clickhouse-server daemon clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions)
|
||||
|
||||
add_library(clickhouse-local LocalServer.cpp)
|
||||
target_link_libraries(clickhouse-local dbms clickhouse_functions clickhouse_aggregate_functions)
|
||||
target_link_libraries(clickhouse-local dbms clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions)
|
||||
|
||||
add_library(clickhouse-extract-from-config ExtractFromConfig.cpp)
|
||||
target_link_libraries(clickhouse-extract-from-config dbms)
|
||||
|
@ -23,6 +23,7 @@
|
||||
#include "StatusFile.h"
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -268,6 +269,7 @@ try
|
||||
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
|
||||
/// Maybe useless
|
||||
if (config().has("macros"))
|
||||
|
@ -40,6 +40,7 @@
|
||||
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -215,6 +216,7 @@ int Server::main(const std::vector<std::string> & args)
|
||||
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions();
|
||||
|
||||
/** Context contains all that query execution is dependent:
|
||||
* settings, available functions, data types, aggregate functions, databases...
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DataStreams/LimitBlockInputStream.h>
|
||||
#include <Storages/System/StorageSystemNumbers.h>
|
||||
|
||||
|
||||
@ -48,8 +49,8 @@ private:
|
||||
};
|
||||
|
||||
|
||||
StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_)
|
||||
: name(name_), columns{{"number", std::make_shared<DataTypeUInt64>()}}, multithreaded(multithreaded_)
|
||||
StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_, size_t limit_)
|
||||
: name(name_), columns{{"number", std::make_shared<DataTypeUInt64>()}}, multithreaded(multithreaded_), limit(limit_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -59,19 +60,30 @@ BlockInputStreams StorageSystemNumbers::read(
|
||||
const ASTPtr & query,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum & processed_stage,
|
||||
const size_t max_block_size,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams)
|
||||
{
|
||||
check(column_names);
|
||||
processed_stage = QueryProcessingStage::FetchColumns;
|
||||
|
||||
if (limit && limit < max_block_size)
|
||||
{
|
||||
max_block_size = std::min(max_block_size, limit);
|
||||
multithreaded = false;
|
||||
}
|
||||
|
||||
if (!multithreaded)
|
||||
num_streams = 1;
|
||||
|
||||
BlockInputStreams res(num_streams);
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
{
|
||||
res[i] = std::make_shared<NumbersBlockInputStream>(max_block_size, i * max_block_size, num_streams * max_block_size);
|
||||
|
||||
if (limit) /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly.
|
||||
res[i] = std::make_shared<LimitBlockInputStream>(res[i], limit * (i + 1) / num_streams - limit * i / num_streams, 0);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -13,6 +13,11 @@ class Context;
|
||||
/** Implements a repository for the system table Numbers.
|
||||
* The table contains the only column number UInt64.
|
||||
* From this table, you can read all natural numbers, starting from 0 (to 2^64 - 1, and then again).
|
||||
*
|
||||
* You could also specify a limit (how many numbers to give).
|
||||
* If multithreaded is specified, numbers will be generated in several streams
|
||||
* (and result could be out of order). If both multithreaded and limit are specified,
|
||||
* the table could give you not exactly 1..limit range, but some arbitary 'limit' numbers.
|
||||
*/
|
||||
class StorageSystemNumbers : public ext::shared_ptr_helper<StorageSystemNumbers>, public IStorage
|
||||
{
|
||||
@ -35,8 +40,10 @@ private:
|
||||
const std::string name;
|
||||
NamesAndTypesList columns;
|
||||
bool multithreaded;
|
||||
size_t limit;
|
||||
|
||||
StorageSystemNumbers(const std::string & name_, bool multithreaded_);
|
||||
/// limit: 0 means unlimited.
|
||||
StorageSystemNumbers(const std::string & name_, bool multithreaded_, size_t limit_ = 0);
|
||||
};
|
||||
|
||||
}
|
||||
|
7
dbms/src/TableFunctions/CMakeLists.txt
Normal file
7
dbms/src/TableFunctions/CMakeLists.txt
Normal file
@ -0,0 +1,7 @@
|
||||
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
|
||||
add_headers_and_sources(clickhouse_table_functions .)
|
||||
|
||||
list(REMOVE_ITEM clickhouse_table_functions_sources TableFunctionFactory.cpp)
|
||||
list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFunctionFactory.h)
|
||||
|
||||
add_library(clickhouse_table_functions ${clickhouse_table_functions_sources})
|
@ -1,10 +1,6 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionMerge.h>
|
||||
#include <TableFunctions/TableFunctionRemote.h>
|
||||
#include <TableFunctions/TableFunctionShardByHash.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
@ -25,11 +21,10 @@ TableFunctionPtr TableFunctionFactory::get(
|
||||
if (context.getSettings().limits.readonly == 1) /** For example, for readonly = 2 - allowed. */
|
||||
throw Exception("Table functions are forbidden in readonly mode", ErrorCodes::READONLY);
|
||||
|
||||
if (name == "merge") return std::make_shared<TableFunctionMerge>();
|
||||
else if (name == "remote") return std::make_shared<TableFunctionRemote>();
|
||||
else if (name == "shardByHash") return std::make_shared<TableFunctionShardByHash>();
|
||||
else
|
||||
auto it = functions.find(name);
|
||||
if (it == functions.end())
|
||||
throw Exception("Unknown table function " + name, ErrorCodes::UNKNOWN_FUNCTION);
|
||||
return it->second();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,19 +1,45 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
#include <common/singleton.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
/** Lets you get a table function by its name.
|
||||
*/
|
||||
class TableFunctionFactory
|
||||
class TableFunctionFactory : public Singleton<TableFunctionFactory>
|
||||
{
|
||||
private:
|
||||
/// No std::function, for smaller object size and less indirection.
|
||||
using Creator = TableFunctionPtr(*)();
|
||||
using TableFunctions = std::unordered_map<String, Creator>;
|
||||
|
||||
TableFunctions functions;
|
||||
|
||||
public:
|
||||
TableFunctionPtr get(
|
||||
const std::string & name,
|
||||
const String & name,
|
||||
const Context & context) const;
|
||||
|
||||
/// Register a table function by its name.
|
||||
template <typename Function>
|
||||
void registerFunction()
|
||||
{
|
||||
if (!functions.emplace(Function::name, []{ return TableFunctionPtr(std::make_unique<Function>()); }).second)
|
||||
throw Exception("TableFunctionFactory: the table function name '" + String(Function::name) + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <TableFunctions/TableFunctionMerge.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -58,14 +59,14 @@ StoragePtr TableFunctionMerge::execute(const ASTPtr & ast_function, const Contex
|
||||
ASTs & args_func = typeid_cast<ASTFunction &>(*ast_function).children;
|
||||
|
||||
if (args_func.size() != 1)
|
||||
throw Exception("Storage Merge requires exactly 2 parameters"
|
||||
throw Exception("Table function 'merge' requires exactly 2 arguments"
|
||||
" - name of source database and regexp for table names.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.at(0)).children;
|
||||
|
||||
if (args.size() != 2)
|
||||
throw Exception("Storage Merge requires exactly 2 parameters"
|
||||
throw Exception("Table function 'merge' requires exactly 2 arguments"
|
||||
" - name of source database and regexp for table names.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
@ -85,4 +86,10 @@ StoragePtr TableFunctionMerge::execute(const ASTPtr & ast_function, const Contex
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionMerge(TableFunctionFactory & factory)
|
||||
{
|
||||
TableFunctionFactory::instance().registerFunction<TableFunctionMerge>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -10,10 +10,11 @@ namespace DB
|
||||
* The structure of the table is taken from the first table that came up, suitable for regexp.
|
||||
* If there is no such table, an exception is thrown.
|
||||
*/
|
||||
class TableFunctionMerge: public ITableFunction
|
||||
class TableFunctionMerge : public ITableFunction
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "merge"; }
|
||||
static constexpr auto name = "merge";
|
||||
std::string getName() const override { return name; }
|
||||
StoragePtr execute(const ASTPtr & ast_function, const Context & context) const override;
|
||||
};
|
||||
|
||||
|
49
dbms/src/TableFunctions/TableFunctionNumbers.cpp
Normal file
49
dbms/src/TableFunctions/TableFunctionNumbers.cpp
Normal file
@ -0,0 +1,49 @@
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionNumbers.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Storages/System/StorageSystemNumbers.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
StoragePtr TableFunctionNumbers::execute(const ASTPtr & ast_function, const Context & context) const
|
||||
{
|
||||
ASTs & args_func = typeid_cast<ASTFunction &>(*ast_function).children;
|
||||
|
||||
if (args_func.size() != 1)
|
||||
throw Exception("Table function 'numbers' requires exactly one argument: amount of numbers.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.at(0)).children;
|
||||
|
||||
if (args.size() != 1)
|
||||
throw Exception("Table function 'numbers' requires exactly one argument: amount of numbers.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
args[0] = evaluateConstantExpressionOrIdentidierAsLiteral(args[0], context);
|
||||
|
||||
UInt64 limit = static_cast<const ASTLiteral &>(*args[0]).value.safeGet<UInt64>();
|
||||
|
||||
auto res = StorageSystemNumbers::create(getName(), false, limit);
|
||||
res->startup();
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionNumbers(TableFunctionFactory & factory)
|
||||
{
|
||||
TableFunctionFactory::instance().registerFunction<TableFunctionNumbers>();
|
||||
}
|
||||
|
||||
}
|
22
dbms/src/TableFunctions/TableFunctionNumbers.h
Normal file
22
dbms/src/TableFunctions/TableFunctionNumbers.h
Normal file
@ -0,0 +1,22 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* numbers(limit)
|
||||
* - the same as SELECT number FROM system.numbers LIMIT limit.
|
||||
* Used for testing purposes, as a simple example of table function.
|
||||
*/
|
||||
class TableFunctionNumbers : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "numbers";
|
||||
std::string getName() const override { return name; }
|
||||
StoragePtr execute(const ASTPtr & ast_function, const Context & context) const override;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionRemote.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -34,7 +35,7 @@ static void append(std::vector<String> & to, const std::vector<String> & what, s
|
||||
}
|
||||
|
||||
if (what.size() * to.size() > max_addresses)
|
||||
throw Exception("Storage Distributed, first argument generates too many result addresses",
|
||||
throw Exception("Table function 'remote': first argument generates too many result addresses",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
std::vector<String> res;
|
||||
for (size_t i = 0; i < to.size(); ++i)
|
||||
@ -107,29 +108,29 @@ static std::vector<String> parseDescription(const String & description, size_t l
|
||||
if (cnt == 0) break;
|
||||
}
|
||||
if (cnt != 0)
|
||||
throw Exception("Storage Distributed, incorrect brace sequence in first argument",
|
||||
throw Exception("Table function 'remote': incorrect brace sequence in first argument",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
/// The presence of a dot - numeric interval
|
||||
if (last_dot != -1)
|
||||
{
|
||||
size_t left, right;
|
||||
if (description[last_dot - 1] != '.')
|
||||
throw Exception("Storage Distributed, incorrect argument in braces (only one dot): " + description.substr(i, m - i + 1),
|
||||
throw Exception("Table function 'remote': incorrect argument in braces (only one dot): " + description.substr(i, m - i + 1),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (!parseNumber(description, i + 1, last_dot - 1, left))
|
||||
throw Exception("Storage Distributed, incorrect argument in braces (Incorrect left number): "
|
||||
throw Exception("Table function 'remote': incorrect argument in braces (Incorrect left number): "
|
||||
+ description.substr(i, m - i + 1),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (!parseNumber(description, last_dot + 1, m, right))
|
||||
throw Exception("Storage Distributed, incorrect argument in braces (Incorrect right number): "
|
||||
throw Exception("Table function 'remote': incorrect argument in braces (Incorrect right number): "
|
||||
+ description.substr(i, m - i + 1),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (left > right)
|
||||
throw Exception("Storage Distributed, incorrect argument in braces (left number is greater then right): "
|
||||
throw Exception("Table function 'remote': incorrect argument in braces (left number is greater then right): "
|
||||
+ description.substr(i, m - i + 1),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (right - left + 1 > max_addresses)
|
||||
throw Exception("Storage Distributed, first argument generates too many result addresses",
|
||||
throw Exception("Table function 'remote': first argument generates too many result addresses",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
bool add_leading_zeroes = false;
|
||||
size_t len = last_dot - 1 - (i + 1);
|
||||
@ -172,7 +173,7 @@ static std::vector<String> parseDescription(const String & description, size_t l
|
||||
|
||||
res.insert(res.end(), cur.begin(), cur.end());
|
||||
if (res.size() > max_addresses)
|
||||
throw Exception("Storage Distributed, first argument generates too many result addresses",
|
||||
throw Exception("Table function 'remote': first argument generates too many result addresses",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return res;
|
||||
@ -285,4 +286,10 @@ StoragePtr TableFunctionRemote::execute(const ASTPtr & ast_function, const Conte
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionRemote(TableFunctionFactory & factory)
|
||||
{
|
||||
TableFunctionFactory::instance().registerFunction<TableFunctionRemote>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -15,7 +15,8 @@ namespace DB
|
||||
class TableFunctionRemote : public ITableFunction
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "remote"; }
|
||||
static constexpr auto name = "remote";
|
||||
std::string getName() const override { return name; }
|
||||
StoragePtr execute(const ASTPtr & ast_function, const Context & context) const override;
|
||||
};
|
||||
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/getClusterName.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <TableFunctions/TableFunctionShardByHash.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -82,4 +83,10 @@ StoragePtr TableFunctionShardByHash::execute(const ASTPtr & ast_function, const
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionShardByHash(TableFunctionFactory & factory)
|
||||
{
|
||||
TableFunctionFactory::instance().registerFunction<TableFunctionShardByHash>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -14,7 +14,8 @@ namespace DB
|
||||
class TableFunctionShardByHash : public ITableFunction
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "shardByHash"; }
|
||||
static constexpr auto name = "shardByHash";
|
||||
std::string getName() const override { return name; }
|
||||
StoragePtr execute(const ASTPtr & ast_function, const Context & context) const override;
|
||||
};
|
||||
|
||||
|
24
dbms/src/TableFunctions/registerTableFunctions.cpp
Normal file
24
dbms/src/TableFunctions/registerTableFunctions.cpp
Normal file
@ -0,0 +1,24 @@
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerTableFunctionMerge(TableFunctionFactory & factory);
|
||||
void registerTableFunctionRemote(TableFunctionFactory & factory);
|
||||
void registerTableFunctionShardByHash(TableFunctionFactory & factory);
|
||||
void registerTableFunctionNumbers(TableFunctionFactory & factory);
|
||||
|
||||
|
||||
void registerTableFunctions()
|
||||
{
|
||||
auto & factory = TableFunctionFactory::instance();
|
||||
|
||||
registerTableFunctionMerge(factory);
|
||||
registerTableFunctionRemote(factory);
|
||||
registerTableFunctionShardByHash(factory);
|
||||
registerTableFunctionNumbers(factory);
|
||||
}
|
||||
|
||||
}
|
8
dbms/src/TableFunctions/registerTableFunctions.h
Normal file
8
dbms/src/TableFunctions/registerTableFunctions.h
Normal file
@ -0,0 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerTableFunctions();
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user