Merge pull request #11847 from ClickHouse/DimasKovas-arch-chooser

Merging #10058
This commit is contained in:
alexey-milovidov 2020-06-22 14:01:10 +03:00 committed by GitHub
commit d6abf45a2d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 1062 additions and 55 deletions

View File

@ -493,8 +493,9 @@ namespace ErrorCodes
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN = 524;
extern const int INCORRECT_DISK_INDEX = 525;
extern const int UNKNOWN_VOLUME_TYPE = 526;
extern const int CASSANDRA_INTERNAL_ERROR = 527;
extern const int NOT_A_LEADER = 528;
extern const int NO_SUITABLE_FUNCTION_IMPLEMENTATION = 527;
extern const int CASSANDRA_INTERNAL_ERROR = 528;
extern const int NOT_A_LEADER = 529;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -120,7 +120,7 @@ void CompressionCodecFactory::registerCompressionCodecWithType(
if (byte_code)
if (!family_code_with_codec.emplace(*byte_code, creator).second)
throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
throw Exception("CompressionCodecFactory: the codec family code '" + std::to_string(*byte_code) + "' is not unique", ErrorCodes::LOGICAL_ERROR);
}
void CompressionCodecFactory::registerCompressionCodec(const String & family_name, std::optional<uint8_t> byte_code, Creator creator)

View File

@ -377,6 +377,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \
M(SettingSeconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \
M(SettingBool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \
M(SettingString, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \
\
M(SettingBool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \
M(SettingBool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \

View File

@ -330,7 +330,7 @@ using SettingLogQueriesType = SettingEnum<QueryLogElementType>;
enum class SettingsBinaryFormat
{
OLD, /// Part of the settings are serialized as strings, and other part as varints. This is the old behaviour.
OLD, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour.
STRINGS, /// All settings are serialized as strings. Before each value the flag `is_ignorable` is serialized.
DEFAULT = STRINGS,
};

View File

@ -96,6 +96,13 @@ if(USE_RAPIDJSON)
target_include_directories(clickhouse_functions SYSTEM PRIVATE ${RAPIDJSON_INCLUDE_DIR})
endif()
option(ENABLE_MULTITARGET_CODE "" ON)
if (ENABLE_MULTITARGET_CODE)
add_definitions(-DENABLE_MULTITARGET_CODE=1)
else()
add_definitions(-DENABLE_MULTITARGET_CODE=0)
endif()
add_subdirectory(GatherUtils)
target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_gatherutils)

View File

@ -1,12 +1,13 @@
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/GatherUtils/GatherUtils.h>
#include <Functions/GatherUtils/Sources.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/PerformanceAdaptors.h>
#include <Functions/TargetSpecific.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnString.h>
namespace DB
{
@ -27,15 +28,13 @@ struct NameEndsWith
static constexpr auto name = "endsWith";
};
DECLARE_MULTITARGET_CODE(
template <typename Name>
class FunctionStartsEndsWith : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionStartsEndsWith>();
}
String getName() const override
{
@ -136,4 +135,41 @@ private:
}
};
) // DECLARE_MULTITARGET_CODE
template <typename Name>
class FunctionStartsEndsWith : public TargetSpecific::Default::FunctionStartsEndsWith<Name>
{
public:
explicit FunctionStartsEndsWith(const Context & context) : selector(context)
{
selector.registerImplementation<TargetArch::Default,
TargetSpecific::Default::FunctionStartsEndsWith<Name>>();
#if USE_MULTITARGET_CODE
selector.registerImplementation<TargetArch::SSE42,
TargetSpecific::SSE42::FunctionStartsEndsWith<Name>>();
selector.registerImplementation<TargetArch::AVX,
TargetSpecific::AVX::FunctionStartsEndsWith<Name>>();
selector.registerImplementation<TargetArch::AVX2,
TargetSpecific::AVX2::FunctionStartsEndsWith<Name>>();
selector.registerImplementation<TargetArch::AVX512F,
TargetSpecific::AVX512F::FunctionStartsEndsWith<Name>>();
#endif
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
selector.selectAndExecute(block, arguments, result, input_rows_count);
}
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionStartsEndsWith<Name>>(context);
}
private:
ImplementationSelector<IFunction> selector;
};
}

View File

@ -40,6 +40,8 @@
#include <Columns/ColumnTuple.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/TargetSpecific.h>
#include <Functions/PerformanceAdaptors.h>
#include <ext/range.h>
#include <ext/bit_cast.h>
@ -573,12 +575,13 @@ public:
};
DECLARE_MULTITARGET_CODE(
template <typename Impl, typename Name>
class FunctionIntHash : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIntHash>(); }
private:
using ToType = typename Impl::ReturnType;
@ -646,13 +649,46 @@ public:
}
};
) // DECLARE_MULTITARGET_CODE
template <typename Impl, typename Name>
class FunctionIntHash : public TargetSpecific::Default::FunctionIntHash<Impl, Name>
{
public:
explicit FunctionIntHash(const Context & context) : selector(context)
{
selector.registerImplementation<TargetArch::Default,
TargetSpecific::Default::FunctionIntHash<Impl, Name>>();
#if USE_MULTITARGET_CODE
selector.registerImplementation<TargetArch::AVX2,
TargetSpecific::AVX2::FunctionIntHash<Impl, Name>>();
selector.registerImplementation<TargetArch::AVX512F,
TargetSpecific::AVX512F::FunctionIntHash<Impl, Name>>();
#endif
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
selector.selectAndExecute(block, arguments, result, input_rows_count);
}
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionIntHash>(context);
}
private:
ImplementationSelector<IFunction> selector;
};
DECLARE_MULTITARGET_CODE(
template <typename Impl>
class FunctionAnyHash : public IFunction
{
public:
static constexpr auto name = Impl::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionAnyHash>(); }
private:
using ToType = typename Impl::ReturnType;
@ -939,6 +975,39 @@ public:
}
};
) // DECLARE_MULTITARGET_CODE
template <typename Impl>
class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash<Impl>
{
public:
explicit FunctionAnyHash(const Context & context) : selector(context)
{
selector.registerImplementation<TargetArch::Default,
TargetSpecific::Default::FunctionAnyHash<Impl>>();
#if USE_MULTITARGET_CODE
selector.registerImplementation<TargetArch::AVX2,
TargetSpecific::AVX2::FunctionAnyHash<Impl>>();
selector.registerImplementation<TargetArch::AVX512F,
TargetSpecific::AVX512F::FunctionAnyHash<Impl>>();
#endif
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
selector.selectAndExecute(block, arguments, result, input_rows_count);
}
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionAnyHash>(context);
}
private:
ImplementationSelector<IFunction> selector;
};
struct URLHashImpl
{

View File

@ -1,8 +1,12 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsRandom.h>
#include <Functions/FunctionFactory.h>
#include <Functions/VectorExtension.h>
#include <Common/HashTable/Hash.h>
#include <Common/randomSeed.h>
#include <common/unaligned.h>
#if USE_MULTITARGET_CODE
# include <x86intrin.h>
#endif
namespace DB
@ -37,12 +41,33 @@ namespace
}
};
void seed(LinearCongruentialGenerator & generator, intptr_t additional_seed)
UInt64 calcSeed(UInt64 rand_seed, UInt64 additional_seed)
{
generator.seed(intHash64(randomSeed() ^ intHash64(additional_seed)));
return intHash64(rand_seed ^ intHash64(additional_seed));
}
void seed(LinearCongruentialGenerator & generator, UInt64 rand_seed, intptr_t additional_seed)
{
generator.seed(calcSeed(rand_seed, additional_seed));
}
/// The array of random numbers from 'head -c8 /dev/urandom | xxd -p'.
/// Can be used for creating seeds for random generators.
constexpr std::array<UInt64, 32> random_numbers = {
0x0c8ff307dabc0c4cULL, 0xf4bce78bf3821c1bULL, 0x4eb628a1e189c21aULL, 0x85ae000d253e0dbcULL,
0xc98073e6480f8a10ULL, 0xb17e9b70a084d570ULL, 0x1361c752b768da8cULL, 0x3d915f60c06d144dULL,
0xd5bc9b7aced79587ULL, 0x66c28000ba8a66cfULL, 0x0fb58da7a48820f5ULL, 0x540ee1b57aa861a1ULL,
0x212f11936ef2db04ULL, 0xa3939cd900edcc58ULL, 0xc676c84420170102ULL, 0xcbdc824e8b4bf3edULL,
0x8296f9d93cc94e3bULL, 0x78a7e826d62085b2ULL, 0xaa30620211fc6c69ULL, 0xbd38de52f0a93677ULL,
0x19983de8d79dcc4eULL, 0x8afe883ef2199e6fULL, 0xb7160f7ed022b60aULL, 0x2ce173d373ddafd4ULL,
0x15762761bb55b9acULL, 0x3e448fc94fdd28e7ULL, 0xa5121232adfbe70aULL, 0xb1e0f6d286112804ULL,
0x6062e96de9554806ULL, 0xcc679b329c28882aULL, 0x5c6d29f45cbc060eULL, 0x1af1325a86ffb162ULL,
};
}
DECLARE_DEFAULT_CODE(
void RandImpl::execute(char * output, size_t size)
{
LinearCongruentialGenerator generator0;
@ -50,10 +75,12 @@ void RandImpl::execute(char * output, size_t size)
LinearCongruentialGenerator generator2;
LinearCongruentialGenerator generator3;
seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast<intptr_t>(output));
seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast<intptr_t>(output));
seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast<intptr_t>(output));
seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast<intptr_t>(output));
UInt64 rand_seed = randomSeed();
seed(generator0, rand_seed, random_numbers[0] + reinterpret_cast<intptr_t>(output));
seed(generator1, rand_seed, random_numbers[1] + reinterpret_cast<intptr_t>(output));
seed(generator2, rand_seed, random_numbers[2] + reinterpret_cast<intptr_t>(output));
seed(generator3, rand_seed, random_numbers[3] + reinterpret_cast<intptr_t>(output));
for (const char * end = output + size; output < end; output += 16)
{
@ -62,8 +89,100 @@ void RandImpl::execute(char * output, size_t size)
unalignedStore<UInt32>(output + 8, generator2.next());
unalignedStore<UInt32>(output + 12, generator3.next());
}
/// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end.
}
) // DECLARE_DEFAULT_CODE
DECLARE_AVX2_SPECIFIC_CODE(
using namespace VectorExtension;
/* Takes 2 vectors with LinearCongruentialGenerator states and combines them into vector with random values.
* From every rand-state we use only bits 15...47 to generate random vector.
*/
inline UInt64x4 combineValues(UInt64x4 a, UInt64x4 b)
{
auto xa = reinterpret_cast<__m256i>(a);
auto xb = reinterpret_cast<__m256i>(b);
/// Every state is 8-byte value and we need to use only 4 from the middle.
/// Swap the low half and the high half of every state to move these bytes from the middle to sides.
/// xa = xa[1, 0, 3, 2, 5, 4, 7, 6]
xa = _mm256_shuffle_epi32(xa, 0xb1);
/// Now every 8-byte value in xa is xx....xx and every value in xb is ..xxxx.. where x is random byte we want to use.
/// Just blend them to get the result vector.
/// result = xa[0],xb[1,2],xa[3,4],xb[5,6],xa[7,8],xb[9,10],xa[11,12],xb[13,14],xa[15]
__m256i result = _mm256_blend_epi16(xa, xb, 0x66);
return reinterpret_cast<UInt64x4>(result);
}
void RandImpl::execute(char * output, size_t size)
{
if (size == 0)
return;
char * end = output + size;
constexpr int vec_size = 4;
constexpr int safe_overwrite = 15;
constexpr int bytes_per_write = 4 * sizeof(UInt64x4);
UInt64 rand_seed = randomSeed();
UInt64 a = LinearCongruentialGenerator::a;
// TODO(dakovalkov): try to remove this.
/// Note: GCC likes to expand multiplication by a constant into shifts + additions.
/// In this case a few multiplications become tens of shifts and additions. That leads to a huge slow down.
/// To avoid it we pretend that 'a' is not a constant. Actually we hope that rand_seed is never 0.
if (rand_seed == 0)
a = LinearCongruentialGenerator::a + 2;
constexpr UInt64 c = LinearCongruentialGenerator::c;
UInt64x4 gens1{};
UInt64x4 gens2{};
UInt64x4 gens3{};
UInt64x4 gens4{};
for (int i = 0; i < vec_size; ++i)
{
gens1[i] = calcSeed(rand_seed, random_numbers[i] + reinterpret_cast<intptr_t>(output));
gens2[i] = calcSeed(rand_seed, random_numbers[i + vec_size] + reinterpret_cast<intptr_t>(output));
gens3[i] = calcSeed(rand_seed, random_numbers[i + 2 * vec_size] + reinterpret_cast<intptr_t>(output));
gens4[i] = calcSeed(rand_seed, random_numbers[i + 3 * vec_size] + reinterpret_cast<intptr_t>(output));
}
while ((end - output) + safe_overwrite >= bytes_per_write)
{
gens1 = gens1 * a + c;
gens2 = gens2 * a + c;
unalignedStore<UInt64x4>(output, combineValues(gens1, gens2));
gens3 = gens3 * a + c;
gens4 = gens4 * a + c;
unalignedStore<UInt64x4>(output + sizeof(UInt64x4), combineValues(gens3, gens4));
gens1 = gens1 * a + c;
gens2 = gens2 * a + c;
unalignedStore<UInt64x4>(output + 2 * sizeof(UInt64x4), combineValues(gens1, gens2));
gens3 = gens3 * a + c;
gens4 = gens4 * a + c;
unalignedStore<UInt64x4>(output + 3 * sizeof(UInt64x4), combineValues(gens3, gens4));
output += bytes_per_write;
}
// Process tail
while ((end - output) > 0)
{
gens1 = gens1 * a + c;
gens2 = gens2 * a + c;
UInt64x4 values = combineValues(gens1, gens2);
for (int i = 0; i < vec_size && (end - output) > 0; ++i)
{
unalignedStore<UInt64>(output, values[i]);
output += sizeof(UInt64);
}
}
}
) // DECLARE_AVX2_SPECIFIC_CODE
}

View File

@ -3,6 +3,8 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnVector.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/TargetSpecific.h>
#include <Functions/PerformanceAdaptors.h>
#include <IO/WriteHelpers.h>
@ -34,19 +36,21 @@ namespace ErrorCodes
* This means that the timer must be of sufficient resolution to give different values to each block.
*/
DECLARE_MULTITARGET_CODE(
struct RandImpl
{
/// Fill memory with random data. The memory region must be 15-bytes padded.
static void execute(char * output, size_t size);
};
) // DECLARE_MULTITARGET_CODE
template <typename ToType, typename Name>
class FunctionRandom : public IFunction
template <typename RandImpl, typename ToType, typename Name>
class FunctionRandomImpl : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRandom>(); }
String getName() const override
{
@ -83,4 +87,33 @@ public:
}
};
template <typename ToType, typename Name>
class FunctionRandom : public FunctionRandomImpl<TargetSpecific::Default::RandImpl, ToType, Name>
{
public:
explicit FunctionRandom(const Context & context) : selector(context)
{
selector.registerImplementation<TargetArch::Default,
FunctionRandomImpl<TargetSpecific::Default::RandImpl, ToType, Name>>();
#if USE_MULTITARGET_CODE
selector.registerImplementation<TargetArch::AVX2,
FunctionRandomImpl<TargetSpecific::AVX2::RandImpl, ToType, Name>>();
#endif
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
selector.selectAndExecute(block, arguments, result, input_rows_count);
}
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionRandom<ToType, Name>>(context);
}
private:
ImplementationSelector<IFunction> selector;
};
}

View File

@ -5,7 +5,7 @@
/// In order to implement a new function you can choose one of two options:
/// * Implement interface for IFunction (old function interface, which is planned to be removed sometimes)
/// * Implement three interfaces for IExecutableFunctionImpl, IFunctionBaseImpl and IFunctionOverloadResolverImpl
/// Generally saying, IFunction represents a union of tree new interfaces. However, it can't be used for all cases.
/// Generally saying, IFunction represents a union of three new interfaces. However, it can't be used for all cases.
/// Examples:
/// * Function properties may depend on arguments type (e.g. toUInt32(UInt8) is globally monotonic, toUInt32(UInt64) - only on intervals)
/// * In implementation of lambda functions DataTypeFunction needs an functional object with known arguments and return type
@ -194,7 +194,7 @@ using FunctionOverloadResolverImplPtr = std::unique_ptr<IFunctionOverloadResolve
/// Previous function interface.
class IFunction : public std::enable_shared_from_this<IFunction>
class IFunction
{
public:
virtual ~IFunction() = default;

View File

@ -0,0 +1,263 @@
#pragma once
#include <Functions/TargetSpecific.h>
#include <Functions/IFunctionImpl.h>
#include <Common/Stopwatch.h>
#include <Interpreters/Context.h>
#include <mutex>
#include <random>
/* This file contains helper class ImplementationSelector. It makes easier to combine
* several implementations of IFunction/IExecutableFunctionImpl.
*/
namespace DB
{
namespace ErrorCodes
{
extern const int NO_SUITABLE_FUNCTION_IMPLEMENTATION;
}
namespace detail
{
class PerformanceStatistics
{
public:
size_t select(bool considarable)
{
/// We don't need to choose/measure anything if there's only one variant.
if (size() == 1)
return 0;
std::lock_guard guard(lock);
size_t best = 0;
double best_sample = data[0].sample(rng);
for (size_t i = 1; i < data.size(); ++i)
{
double sample = data[i].sample(rng);
if (sample < best_sample)
{
best_sample = sample;
best = i;
}
}
if (considarable)
data[best].run();
return best;
}
void complete(size_t id, double seconds, double bytes)
{
if (size() == 1)
return;
std::lock_guard guard(lock);
data[id].complete(seconds, bytes);
}
size_t size() const
{
return data.size();
}
bool empty() const
{
return size() == 0;
}
void emplace_back()
{
data.emplace_back();
}
private:
struct Element
{
int completed_count = 0;
int running_count = 0;
double sum = 0;
int adjustedCount() const
{
return completed_count - NUM_INVOCATIONS_TO_THROW_OFF;
}
double mean() const
{
return sum / adjustedCount();
}
/// For better convergence, we don't use proper estimate of stddev.
/// We want to eventually separate between two algorithms even in case
/// when there is no statistical significant difference between them.
double sigma() const
{
return mean() / sqrt(adjustedCount());
}
void run()
{
++running_count;
}
void complete(double seconds, double bytes)
{
--running_count;
++completed_count;
if (adjustedCount() > 0)
sum += seconds / bytes;
}
double sample(pcg64 & stat_rng) const
{
/// If there is a variant with not enough statistics, always choose it.
/// And in that case prefer variant with less number of invocations.
if (adjustedCount() < 2)
return adjustedCount() - 1 + running_count;
return std::normal_distribution<>(mean(), sigma())(stat_rng);
}
};
std::vector<Element> data;
std::mutex lock;
/// It's Ok that generator is not seeded.
pcg64 rng;
/// Cold invocations may be affected by additional memory latencies. Don't take first invocations into account.
static constexpr int NUM_INVOCATIONS_TO_THROW_OFF = 2;
};
template <typename T, class = decltype(T::getImplementationTag())>
std::true_type hasImplementationTagTest(const T&);
std::false_type hasImplementationTagTest(...);
template <typename T>
constexpr bool has_implementation_tag = decltype(hasImplementationTagTest(std::declval<T>()))::value;
/* Implementation tag is used to run specific implementation (for debug/testing purposes).
* It can be specified via static method ::getImplementationTag() in Function (optional).
*/
template <typename T>
String getImplementationTag(TargetArch arch)
{
if constexpr (has_implementation_tag<T>)
return toString(arch) + "_" + T::getImplementationTag();
else
return toString(arch);
}
}
/* Class which is used to store implementations for the function and to select the best one to run
* based on processor architecture and statistics from previous runs.
*
* FunctionInterface is typically IFunction or IExecutableFunctionImpl, but practically it can be
* any interface that contains "execute" method (IFunction is an exception and is supported as well).
*
* Example of usage:
*
* class MyDefaulImpl : public IFunction {...};
* DECLARE_AVX2_SPECIFIC_CODE(
* class MyAVX2Impl : public IFunction {...};
* )
*
* /// All methods but execute/executeImpl are usually not bottleneck, so just use them from
* /// default implementation.
* class MyFunction : public MyDefaultImpl
* {
* MyFunction(const Context & context) : selector(context) {
* /// Register all implementations in constructor.
* /// There could be as many implementation for every target as you want.
* selector.registerImplementation<TargetArch::Default, MyDefaultImpl>();
* #if USE_MULTITARGET_CODE
* selector.registreImplementation<TargetArch::AVX2, TargetSpecific::AVX2::MyAVX2Impl>();
* #endif
* }
*
* void executeImpl(...) override {
* selector.selectAndExecute(...);
* }
*
* static FunctionPtr create(const Context & context) {
* return std::make_shared<MyFunction>(context);
* }
* private:
* ImplementationSelector<IFunction> selector;
* };
*/
template <typename FunctionInterface>
class ImplementationSelector
{
public:
using ImplementationPtr = std::shared_ptr<FunctionInterface>;
ImplementationSelector(const Context & context_) : context(context_) {}
/* Select the best implementation based on previous runs.
* If FunctionInterface is IFunction, then "executeImpl" method of the implementation will be called
* and "execute" otherwise.
*/
void selectAndExecute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
if (implementations.empty())
throw Exception("There are no available implementations for function " "TODO(dakovalkov): add name",
ErrorCodes::NO_SUITABLE_FUNCTION_IMPLEMENTATION);
/// Statistics shouldn't rely on small blocks.
bool considerable = (input_rows_count > 1000);
size_t id = statistics.select(considerable);
Stopwatch watch;
if constexpr (std::is_same_v<FunctionInterface, IFunction>)
implementations[id]->executeImpl(block, arguments, result, input_rows_count);
else
implementations[id]->execute(block, arguments, result, input_rows_count);
watch.stop();
if (considerable)
{
// TODO(dakovalkov): Calculate something more informative than rows count.
statistics.complete(id, watch.elapsedSeconds(), input_rows_count);
}
}
/* Register new implementation for function.
*
* Arch - required instruction set for running the implementation. It's guarantied that no method would
* be called (even the constructor and static methods) if the processor doesn't support this instruction set.
*
* FunctionImpl - implementation, should be inherited from template argument FunctionInterface.
*
* All function arguments will be forwarded to the implementation constructor.
*/
template <TargetArch Arch, typename FunctionImpl, typename ...Args>
void registerImplementation(Args &&... args)
{
if (isArchSupported(Arch))
{
// TODO(dakovalkov): make this option better.
const auto & choose_impl = context.getSettingsRef().function_implementation.value;
if (choose_impl.empty() || choose_impl == detail::getImplementationTag<FunctionImpl>(Arch))
{
implementations.emplace_back(std::make_shared<FunctionImpl>(std::forward<Args>(args)...));
statistics.emplace_back();
}
}
}
private:
const Context & context;
std::vector<ImplementationPtr> implementations;
detail::PerformanceStatistics statistics;
};
}

View File

@ -0,0 +1,42 @@
#include <Functions/TargetSpecific.h>
#include <Common/CpuId.h>
namespace DB
{
UInt32 getSupportedArchs()
{
UInt32 result = 0;
if (Cpu::CpuFlagsCache::have_SSE42)
result |= static_cast<UInt32>(TargetArch::SSE42);
if (Cpu::CpuFlagsCache::have_AVX)
result |= static_cast<UInt32>(TargetArch::AVX);
if (Cpu::CpuFlagsCache::have_AVX2)
result |= static_cast<UInt32>(TargetArch::AVX2);
if (Cpu::CpuFlagsCache::have_AVX512F)
result |= static_cast<UInt32>(TargetArch::AVX512F);
return result;
}
bool isArchSupported(TargetArch arch)
{
static UInt32 arches = getSupportedArchs();
return arch == TargetArch::Default || (arches & static_cast<UInt32>(arch));
}
String toString(TargetArch arch)
{
switch (arch)
{
case TargetArch::Default: return "default";
case TargetArch::SSE42: return "sse42";
case TargetArch::AVX: return "avx";
case TargetArch::AVX2: return "avx2";
case TargetArch::AVX512F: return "avx512f";
}
__builtin_unreachable();
}
}

View File

@ -0,0 +1,213 @@
#pragma once
#include <Core/Types.h>
/* This file contains macros and helpers for writing platform-dependent code.
*
* Macros DECLARE_<Arch>_SPECIFIC_CODE will wrap code inside it into the
* namespace TargetSpecific::<Arch> and enable Arch-specific compile options.
* Thus, it's allowed to call functions inside these namespaces only after
* checking platform in runtime (see isArchSupported() below).
*
* If compiler is not gcc/clang or target isn't x86_64 or ENABLE_MULTITARGET_CODE
* was set to OFF in cmake, all code inside these macroses will be removed and
* USE_MUTLITARGE_CODE will be set to 0. Use #if USE_MUTLITARGE_CODE whenever you
* use anything from this namespaces.
*
* For similarities there is a macros DECLARE_DEFAULT_CODE, which wraps code
* into the namespace TargetSpecific::Default but dosn't specify any additional
* copile options. Functions and classes inside this macros are available regardless
* of USE_MUTLITARGE_CODE.
*
* Example of usage:
*
* DECLARE_DEFAULT_CODE (
* int funcImpl() {
* return 1;
* }
* ) // DECLARE_DEFAULT_CODE
*
* DECLARE_AVX2_SPECIFIC_CODE (
* int funcImpl() {
* return 2;
* }
* ) // DECLARE_DEFAULT_CODE
*
* int func() {
* #if USE_MULTITARGET_CODE
* if (isArchSupported(TargetArch::AVX2))
* return TargetSpecifc::AVX2::funcImpl();
* #endif
* return TargetSpecifc::Default::funcImpl();
* }
*
* Sometimes code may benefit from compiling with different options.
* For these purposes use DECLARE_MULTITARGET_CODE macros. It will create a copy
* of the code for every supported target and compile it with different options.
* These copies are available via TargetSpecifc namespaces described above.
*
* Inside every TargetSpecific namespace there is a constexpr variable BuildArch,
* which indicates the target platform for current code.
*
* Example:
*
* DECLARE_MULTITARGET_CODE(
* int funcImpl(int size, ...) {
* int iteration_size = 1;
* if constexpr (BuildArch == TargetArch::SSE42)
* iteration_size = 2
* else if constexpr (BuildArch == TargetArch::AVX || BuildArch == TargetArch::AVX2)
* iteration_size = 4;
* for (int i = 0; i < size; i += iteration_size)
* ...
* }
* ) // DECLARE_MULTITARGET_CODE
*
* // All target-specific and default implementations are available here via
* TargetSpecific::<Arch>::funcImpl. Use runtime detection to choose one.
*
* If you want to write IFunction or IExecutableFuncionImpl with several implementations
* see PerformanceAdaptors.h.
*/
namespace DB
{
enum class TargetArch : UInt32
{
Default = 0, /// Without any additional compiler options.
SSE42 = (1 << 0), /// SSE4.2
AVX = (1 << 1),
AVX2 = (1 << 2),
AVX512F = (1 << 3),
};
/// Runtime detection.
bool isArchSupported(TargetArch arch);
String toString(TargetArch arch);
#if ENABLE_MULTITARGET_CODE && defined(__GNUC__) && defined(__x86_64__)
#define USE_MULTITARGET_CODE 1
#if defined(__clang__)
# define BEGIN_AVX512F_SPECIFIC_CODE \
_Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f\"))),apply_to=function)")
# define BEGIN_AVX2_SPECIFIC_CODE \
_Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2\"))),apply_to=function)")
# define BEGIN_AVX_SPECIFIC_CODE \
_Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx\"))),apply_to=function)")
# define BEGIN_SSE42_SPECIFIC_CODE \
_Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt\"))),apply_to=function)")
# define END_TARGET_SPECIFIC_CODE \
_Pragma("clang attribute pop")
/* Clang shows warning when there aren't any objects to apply pragma.
* To prevent this warning we define this function inside every macros with pragmas.
*/
# define DUMMY_FUNCTION_DEFINITION void __dummy_function_definition();
#else
# define BEGIN_AVX512F_SPECIFIC_CODE \
_Pragma("GCC push_options") \
_Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,tune=native\")")
# define BEGIN_AVX2_SPECIFIC_CODE \
_Pragma("GCC push_options") \
_Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,tune=native\")")
# define BEGIN_AVX_SPECIFIC_CODE \
_Pragma("GCC push_options") \
_Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,tune=native\")")
# define BEGIN_SSE42_SPECIFIC_CODE \
_Pragma("GCC push_options") \
_Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,tune=native\")")
# define END_TARGET_SPECIFIC_CODE \
_Pragma("GCC pop_options")
/* GCC doesn't show such warning, we don't need to define anything.
*/
# define DUMMY_FUNCTION_DEFINITION
#endif
#define DECLARE_SSE42_SPECIFIC_CODE(...) \
BEGIN_SSE42_SPECIFIC_CODE \
namespace TargetSpecific::SSE42 { \
DUMMY_FUNCTION_DEFINITION \
using namespace DB::TargetSpecific::SSE42; \
__VA_ARGS__ \
} \
END_TARGET_SPECIFIC_CODE
#define DECLARE_AVX_SPECIFIC_CODE(...) \
BEGIN_AVX_SPECIFIC_CODE \
namespace TargetSpecific::AVX { \
DUMMY_FUNCTION_DEFINITION \
using namespace DB::TargetSpecific::AVX; \
__VA_ARGS__ \
} \
END_TARGET_SPECIFIC_CODE
#define DECLARE_AVX2_SPECIFIC_CODE(...) \
BEGIN_AVX2_SPECIFIC_CODE \
namespace TargetSpecific::AVX2 { \
DUMMY_FUNCTION_DEFINITION \
using namespace DB::TargetSpecific::AVX2; \
__VA_ARGS__ \
} \
END_TARGET_SPECIFIC_CODE
#define DECLARE_AVX512F_SPECIFIC_CODE(...) \
BEGIN_AVX512F_SPECIFIC_CODE \
namespace TargetSpecific::AVX512F { \
DUMMY_FUNCTION_DEFINITION \
using namespace DB::TargetSpecific::AVX512F; \
__VA_ARGS__ \
} \
END_TARGET_SPECIFIC_CODE
#else
#define USE_MULTITARGET_CODE 0
/* Multitarget code is disabled, just delete target-specific code.
*/
#define DECLARE_SSE42_SPECIFIC_CODE(...)
#define DECLARE_AVX_SPECIFIC_CODE(...)
#define DECLARE_AVX2_SPECIFIC_CODE(...)
#define DECLARE_AVX512F_SPECIFIC_CODE(...)
#endif
#define DECLARE_DEFAULT_CODE(...) \
namespace TargetSpecific::Default { \
using namespace DB::TargetSpecific::Default; \
__VA_ARGS__ \
}
#define DECLARE_MULTITARGET_CODE(...) \
DECLARE_DEFAULT_CODE (__VA_ARGS__) \
DECLARE_SSE42_SPECIFIC_CODE (__VA_ARGS__) \
DECLARE_AVX_SPECIFIC_CODE (__VA_ARGS__) \
DECLARE_AVX2_SPECIFIC_CODE (__VA_ARGS__) \
DECLARE_AVX512F_SPECIFIC_CODE(__VA_ARGS__)
DECLARE_DEFAULT_CODE(
constexpr auto BuildArch = TargetArch::Default;
) // DECLARE_DEFAULT_CODE
DECLARE_SSE42_SPECIFIC_CODE(
constexpr auto BuildArch = TargetArch::SSE42;
) // DECLARE_SSE42_SPECIFIC_CODE
DECLARE_AVX_SPECIFIC_CODE(
constexpr auto BuildArch = TargetArch::AVX;
) // DECLARE_AVX_SPECIFIC_CODE
DECLARE_AVX2_SPECIFIC_CODE(
constexpr auto BuildArch = TargetArch::AVX2;
) // DECLARE_AVX2_SPECIFIC_CODE
DECLARE_AVX512F_SPECIFIC_CODE(
constexpr auto BuildArch = TargetArch::AVX512F;
) // DECLARE_AVX512F_SPECIFIC_CODE
}

View File

@ -0,0 +1,78 @@
#pragma once
#include <Core/Types.h>
// Contains types declarations and wrappers for GCC vector extension.
namespace DB::VectorExtension
{
typedef UInt64 UInt64x2 __attribute__ ((vector_size (sizeof(UInt64) * 2)));
typedef UInt64 UInt64x4 __attribute__ ((vector_size (sizeof(UInt64) * 4)));
typedef UInt64 UInt64x8 __attribute__ ((vector_size (sizeof(UInt64) * 8)));
typedef UInt32 UInt32x2 __attribute__ ((vector_size (sizeof(UInt32) * 2)));
typedef UInt32 UInt32x4 __attribute__ ((vector_size (sizeof(UInt32) * 4)));
typedef UInt32 UInt32x8 __attribute__ ((vector_size (sizeof(UInt32) * 8)));
typedef UInt32 UInt32x16 __attribute__ ((vector_size (sizeof(UInt32) * 16)));
typedef UInt16 UInt16x2 __attribute__ ((vector_size (sizeof(UInt16) * 2)));
typedef UInt16 UInt16x4 __attribute__ ((vector_size (sizeof(UInt16) * 4)));
typedef UInt16 UInt16x8 __attribute__ ((vector_size (sizeof(UInt16) * 8)));
typedef UInt16 UInt16x16 __attribute__ ((vector_size (sizeof(UInt16) * 16)));
typedef UInt16 UInt16x32 __attribute__ ((vector_size (sizeof(UInt16) * 32)));
typedef UInt8 UInt8x2 __attribute__ ((vector_size (sizeof(UInt8) * 2)));
typedef UInt8 UInt8x4 __attribute__ ((vector_size (sizeof(UInt8) * 4)));
typedef UInt8 UInt8x8 __attribute__ ((vector_size (sizeof(UInt8) * 8)));
typedef UInt8 UInt8x16 __attribute__ ((vector_size (sizeof(UInt8) * 16)));
typedef UInt8 UInt8x32 __attribute__ ((vector_size (sizeof(UInt8) * 32)));
typedef UInt8 UInt8x64 __attribute__ ((vector_size (sizeof(UInt8) * 64)));
namespace detail
{
template <int Size>
struct DummyStruct;
template <>
struct DummyStruct<4>
{
using UInt8Type = UInt8x4;
using UInt16Type = UInt16x4;
using UInt32Type = UInt32x4;
using UInt64Type = UInt64x4;
};
template <>
struct DummyStruct<8>
{
using UInt8Type = UInt8x8;
using UInt16Type = UInt16x8;
using UInt32Type = UInt32x8;
using UInt64Type = UInt64x8;
};
template <>
struct DummyStruct<16>
{
using UInt8Type = UInt8x16;
using UInt16Type = UInt16x16;
using UInt32Type = UInt32x16;
};
template <>
struct DummyStruct<32>
{
using UInt8Type = UInt8x32;
using UInt16Type = UInt16x32;
};
}
// Same as above via template, e.g. UInt64x<8>
template <int Size>
using UInt8x = typename detail::DummyStruct<Size>::UInt8Type;
template <int Size>
using UInt16x = typename detail::DummyStruct<Size>::UInt16Type;
template <int Size>
using UInt32x = typename detail::DummyStruct<Size>::UInt32Type;
template <int Size>
using UInt64x = typename detail::DummyStruct<Size>::UInt64Type;
}

View File

@ -5,11 +5,16 @@
namespace DB
{
#define DECLARE_SEVERAL_IMPLEMENTATIONS(...) \
DECLARE_DEFAULT_CODE (__VA_ARGS__) \
DECLARE_AVX2_SPECIFIC_CODE(__VA_ARGS__)
DECLARE_SEVERAL_IMPLEMENTATIONS(
class FunctionGenerateUUIDv4 : public IFunction
{
public:
static constexpr auto name = "generateUUIDv4";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGenerateUUIDv4>(); }
String getName() const override
{
@ -32,6 +37,8 @@ public:
size_t size = input_rows_count;
vec_to.resize(size);
/// RandImpl is target-dependent and is not the same in different TargetSpecific namespaces.
RandImpl::execute(reinterpret_cast<char *>(vec_to.data()), vec_to.size() * sizeof(UInt128));
for (UInt128 & uuid: vec_to)
@ -46,6 +53,37 @@ public:
}
};
) // DECLARE_SEVERAL_IMPLEMENTATIONS
#undef DECLARE_SEVERAL_IMPLEMENTATIONS
class FunctionGenerateUUIDv4 : public TargetSpecific::Default::FunctionGenerateUUIDv4
{
public:
explicit FunctionGenerateUUIDv4(const Context & context) : selector(context)
{
selector.registerImplementation<TargetArch::Default,
TargetSpecific::Default::FunctionGenerateUUIDv4>();
#if USE_MULTITARGET_CODE
selector.registerImplementation<TargetArch::AVX2,
TargetSpecific::AVX2::FunctionGenerateUUIDv4>();
#endif
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
selector.selectAndExecute(block, arguments, result, input_rows_count);
}
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionGenerateUUIDv4>(context);
}
private:
ImplementationSelector<IFunction> selector;
};
void registerFunctionGenerateUUIDv4(FunctionFactory & factory)
{
factory.registerFunction<FunctionGenerateUUIDv4>();

View File

@ -6,6 +6,8 @@
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <Functions/TargetSpecific.h>
#include <Functions/PerformanceAdaptors.h>
#include <ext/range.h>
#include <cmath>
@ -153,6 +155,12 @@ enum class Method
WGS84_METERS,
};
}
DECLARE_MULTITARGET_CODE(
namespace
{
template <Method method>
float distance(float lon1deg, float lat1deg, float lon2deg, float lat2deg)
@ -220,7 +228,6 @@ float distance(float lon1deg, float lat1deg, float lon2deg, float lat2deg)
}
template <Method method>
class FunctionGeoDistance : public IFunction
{
@ -230,8 +237,6 @@ public:
: ((method == Method::SPHERE_METERS) ? "greatCircleDistance"
: "geoDistance");
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGeoDistance<method>>(); }
private:
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 4; }
@ -272,6 +277,40 @@ private:
}
};
) // DECLARE_MULTITARGET_CODE
template <Method method>
class FunctionGeoDistance : public TargetSpecific::Default::FunctionGeoDistance<method>
{
public:
explicit FunctionGeoDistance(const Context & context) : selector(context)
{
selector.registerImplementation<TargetArch::Default,
TargetSpecific::Default::FunctionGeoDistance<method>>();
#if USE_MULTITARGET_CODE
selector.registerImplementation<TargetArch::AVX,
TargetSpecific::AVX::FunctionGeoDistance<method>>();
selector.registerImplementation<TargetArch::AVX2,
TargetSpecific::AVX2::FunctionGeoDistance<method>>();
selector.registerImplementation<TargetArch::AVX512F,
TargetSpecific::AVX512F::FunctionGeoDistance<method>>();
#endif
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
selector.selectAndExecute(block, arguments, result, input_rows_count);
}
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionGeoDistance<method>>(context);
}
private:
ImplementationSelector<IFunction> selector;
};
void registerFunctionGeoDistance(FunctionFactory & factory)
{

View File

@ -99,7 +99,8 @@ public:
argument_types.emplace_back(arguments.back().type);
typename ColumnVector<ToType>::Container vec_to(1);
RandImpl::execute(reinterpret_cast<char *>(vec_to.data()), sizeof(ToType));
TargetSpecific::Default::RandImpl::execute(reinterpret_cast<char *>(vec_to.data()), sizeof(ToType));
ToType value = vec_to[0];
return std::make_unique<FunctionBaseRandomConstant<ToType, Name>>(value, argument_types, return_type);

View File

@ -3,6 +3,8 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/PerformanceAdaptors.h>
#include <Functions/FunctionsRandom.h>
#include <pcg_random.hpp>
#include <Common/randomSeed.h>
#include <common/arithmeticOverflow.h>
@ -21,13 +23,12 @@ namespace ErrorCodes
/* Generate random fixed string with fully random bytes (including zero). */
class FunctionRandomFixedString : public IFunction
template <typename RandImpl>
class FunctionRandomFixedStringImpl : public IFunction
{
public:
static constexpr auto name = "randomFixedString";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRandomFixedString>(); }
String getName() const override { return name; }
bool isVariadic() const override { return false; }
@ -68,20 +69,40 @@ public:
/// Fill random bytes.
data_to.resize(total_size);
pcg64_fast rng(randomSeed()); /// TODO It is inefficient. We should use SIMD PRNG instead.
auto * pos = data_to.data();
auto * end = pos + data_to.size();
while (pos < end)
{
unalignedStore<UInt64>(pos, rng());
pos += sizeof(UInt64); // We have padding in column buffers that we can overwrite.
}
RandImpl::execute(reinterpret_cast<char *>(data_to.data()), total_size);
block.getByPosition(result).column = std::move(col_to);
}
};
class FunctionRandomFixedString : public FunctionRandomFixedStringImpl<TargetSpecific::Default::RandImpl>
{
public:
explicit FunctionRandomFixedString(const Context & context) : selector(context)
{
selector.registerImplementation<TargetArch::Default,
FunctionRandomFixedStringImpl<TargetSpecific::Default::RandImpl>>();
#if USE_MULTITARGET_CODE
selector.registerImplementation<TargetArch::AVX2,
FunctionRandomFixedStringImpl<TargetSpecific::AVX2::RandImpl>>();
#endif
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
selector.selectAndExecute(block, arguments, result, input_rows_count);
}
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionRandomFixedString>(context);
}
private:
ImplementationSelector<IFunction> selector;
};
void registerFunctionRandomFixedString(FunctionFactory & factory)
{
factory.registerFunction<FunctionRandomFixedString>();

View File

@ -3,6 +3,8 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionsRandom.h>
#include <Functions/PerformanceAdaptors.h>
#include <pcg_random.hpp>
#include <Common/randomSeed.h>
#include <common/unaligned.h>
@ -19,13 +21,12 @@ namespace ErrorCodes
/* Generate random string of specified length with fully random bytes (including zero). */
class FunctionRandomString : public IFunction
template <typename RandImpl>
class FunctionRandomStringImpl : public IFunction
{
public:
static constexpr auto name = "randomString";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRandomString>(); }
String getName() const override { return name; }
bool isVariadic() const override { return true; }
@ -83,18 +84,10 @@ public:
/// Fill random bytes.
data_to.resize(offsets_to.back());
pcg64_fast rng(randomSeed()); /// TODO It is inefficient. We should use SIMD PRNG instead.
auto * pos = data_to.data();
auto * end = pos + data_to.size();
while (pos < end)
{
unalignedStore<UInt64>(pos, rng());
pos += sizeof(UInt64); // We have padding in column buffers that we can overwrite.
}
RandImpl::execute(reinterpret_cast<char *>(data_to.data()), data_to.size());
/// Put zero bytes in between.
pos = data_to.data();
auto * pos = data_to.data();
for (size_t row_num = 0; row_num < input_rows_count; ++row_num)
pos[offsets_to[row_num] - 1] = 0;
@ -102,6 +95,34 @@ public:
}
};
class FunctionRandomString : public FunctionRandomStringImpl<TargetSpecific::Default::RandImpl>
{
public:
explicit FunctionRandomString(const Context & context) : selector(context)
{
selector.registerImplementation<TargetArch::Default,
FunctionRandomStringImpl<TargetSpecific::Default::RandImpl>>();
#if USE_MULTITARGET_CODE
selector.registerImplementation<TargetArch::AVX2,
FunctionRandomStringImpl<TargetSpecific::AVX2::RandImpl>>();
#endif
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
selector.selectAndExecute(block, arguments, result, input_rows_count);
}
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionRandomString>(context);
}
private:
ImplementationSelector<IFunction> selector;
};
void registerFunctionRandomString(FunctionFactory & factory)
{
factory.registerFunction<FunctionRandomString>();

View File

@ -365,6 +365,7 @@ SRCS(
subtractYears.cpp
tan.cpp
tanh.cpp
TargetSpecific.cpp
tgamma.cpp
throwIf.cpp
timeSlot.cpp

View File

@ -0,0 +1,24 @@
<test>
<stop_conditions>
<all_of>
<total_time_ms>10000</total_time_ms>
</all_of>
<any_of>
<total_time_ms>20000</total_time_ms>
</any_of>
</stop_conditions>
<substitutions>
<substitution>
<name>table</name>
<values>
<value>zeros(100000000)</value>
</values>
</substitution>
</substitutions>
<query>SELECT count() FROM (SELECT rand() FROM {table}) </query>
<query>SELECT count() FROM (SELECT rand64() FROM {table}) </query>
<query>SELECT count() FROM (SELECT generateUUIDv4() FROM {table}) </query>
</test>