diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 00ccb1fa7cc..09f4879c3f4 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -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; diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 5d5c5c14de6..2598fc07b08 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -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 byte_code, Creator creator) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6d856eaf88e..dc50a6a6957 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -377,6 +377,7 @@ struct Settings : public SettingsCollection 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) \ diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index 71a308fb37e..e85baf82fb6 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -330,7 +330,7 @@ using SettingLogQueriesType = SettingEnum; 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, }; diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index be78a42d02b..244d08e104c 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -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) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 4f56a827f4c..69627eb2ead 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -1,12 +1,13 @@ -#include #include #include #include +#include +#include +#include #include #include #include - namespace DB { @@ -27,15 +28,13 @@ struct NameEndsWith static constexpr auto name = "endsWith"; }; +DECLARE_MULTITARGET_CODE( + template class FunctionStartsEndsWith : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } String getName() const override { @@ -136,4 +135,41 @@ private: } }; +) // DECLARE_MULTITARGET_CODE + +template +class FunctionStartsEndsWith : public TargetSpecific::Default::FunctionStartsEndsWith +{ +public: + explicit FunctionStartsEndsWith(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); + #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>(context); + } + +private: + ImplementationSelector selector; +}; + } diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index f647390e1c8..b4c87dd761a 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -40,6 +40,8 @@ #include #include #include +#include +#include #include #include @@ -573,12 +575,13 @@ public: }; +DECLARE_MULTITARGET_CODE( + template class FunctionIntHash : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } private: using ToType = typename Impl::ReturnType; @@ -646,13 +649,46 @@ public: } }; +) // DECLARE_MULTITARGET_CODE + +template +class FunctionIntHash : public TargetSpecific::Default::FunctionIntHash +{ +public: + explicit FunctionIntHash(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + selector.registerImplementation>(); + #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(context); + } + +private: + ImplementationSelector selector; +}; + +DECLARE_MULTITARGET_CODE( template class FunctionAnyHash : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } private: using ToType = typename Impl::ReturnType; @@ -939,6 +975,39 @@ public: } }; +) // DECLARE_MULTITARGET_CODE + +template +class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash +{ +public: + explicit FunctionAnyHash(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + selector.registerImplementation>(); + #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(context); + } + +private: + ImplementationSelector selector; +}; + struct URLHashImpl { diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 19b2f08cdba..9436ce0f5e2 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -1,8 +1,12 @@ -#include #include +#include +#include #include #include #include +#if USE_MULTITARGET_CODE +# include +#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 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(output)); - seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast(output)); - seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast(output)); - seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)); + UInt64 rand_seed = randomSeed(); + + seed(generator0, rand_seed, random_numbers[0] + reinterpret_cast(output)); + seed(generator1, rand_seed, random_numbers[1] + reinterpret_cast(output)); + seed(generator2, rand_seed, random_numbers[2] + reinterpret_cast(output)); + seed(generator3, rand_seed, random_numbers[3] + reinterpret_cast(output)); for (const char * end = output + size; output < end; output += 16) { @@ -62,8 +89,100 @@ void RandImpl::execute(char * output, size_t size) unalignedStore(output + 8, generator2.next()); unalignedStore(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(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(output)); + gens2[i] = calcSeed(rand_seed, random_numbers[i + vec_size] + reinterpret_cast(output)); + gens3[i] = calcSeed(rand_seed, random_numbers[i + 2 * vec_size] + reinterpret_cast(output)); + gens4[i] = calcSeed(rand_seed, random_numbers[i + 3 * vec_size] + reinterpret_cast(output)); + } + + while ((end - output) + safe_overwrite >= bytes_per_write) + { + gens1 = gens1 * a + c; + gens2 = gens2 * a + c; + unalignedStore(output, combineValues(gens1, gens2)); + gens3 = gens3 * a + c; + gens4 = gens4 * a + c; + unalignedStore(output + sizeof(UInt64x4), combineValues(gens3, gens4)); + gens1 = gens1 * a + c; + gens2 = gens2 * a + c; + unalignedStore(output + 2 * sizeof(UInt64x4), combineValues(gens1, gens2)); + gens3 = gens3 * a + c; + gens4 = gens4 * a + c; + unalignedStore(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(output, values[i]); + output += sizeof(UInt64); + } + } +} + +) // DECLARE_AVX2_SPECIFIC_CODE + } diff --git a/src/Functions/FunctionsRandom.h b/src/Functions/FunctionsRandom.h index 1ac6d24a356..b80ddb6f59e 100644 --- a/src/Functions/FunctionsRandom.h +++ b/src/Functions/FunctionsRandom.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include @@ -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 -class FunctionRandom : public IFunction +template +class FunctionRandomImpl : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override { @@ -83,4 +87,33 @@ public: } }; +template +class FunctionRandom : public FunctionRandomImpl +{ +public: + explicit FunctionRandom(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + #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>(context); + } + +private: + ImplementationSelector selector; +}; + } diff --git a/src/Functions/IFunctionImpl.h b/src/Functions/IFunctionImpl.h index 116363705de..27e7aec6bd4 100644 --- a/src/Functions/IFunctionImpl.h +++ b/src/Functions/IFunctionImpl.h @@ -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 +class IFunction { public: virtual ~IFunction() = default; diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h new file mode 100644 index 00000000000..b17be0619a9 --- /dev/null +++ b/src/Functions/PerformanceAdaptors.h @@ -0,0 +1,263 @@ +#pragma once + +#include +#include + +#include +#include + +#include +#include + +/* 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 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 + std::true_type hasImplementationTagTest(const T&); + std::false_type hasImplementationTagTest(...); + + template + constexpr bool has_implementation_tag = decltype(hasImplementationTagTest(std::declval()))::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 + String getImplementationTag(TargetArch arch) + { + if constexpr (has_implementation_tag) + 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(); + * #if USE_MULTITARGET_CODE + * selector.registreImplementation(); + * #endif + * } + * + * void executeImpl(...) override { + * selector.selectAndExecute(...); + * } + * + * static FunctionPtr create(const Context & context) { + * return std::make_shared(context); + * } + * private: + * ImplementationSelector selector; + * }; + */ +template +class ImplementationSelector +{ +public: + using ImplementationPtr = std::shared_ptr; + + 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) + 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 + 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(Arch)) + { + implementations.emplace_back(std::make_shared(std::forward(args)...)); + statistics.emplace_back(); + } + } + } + +private: + const Context & context; + std::vector implementations; + detail::PerformanceStatistics statistics; +}; + +} diff --git a/src/Functions/TargetSpecific.cpp b/src/Functions/TargetSpecific.cpp new file mode 100644 index 00000000000..830611fea7a --- /dev/null +++ b/src/Functions/TargetSpecific.cpp @@ -0,0 +1,42 @@ +#include + +#include + +namespace DB +{ + +UInt32 getSupportedArchs() +{ + UInt32 result = 0; + if (Cpu::CpuFlagsCache::have_SSE42) + result |= static_cast(TargetArch::SSE42); + if (Cpu::CpuFlagsCache::have_AVX) + result |= static_cast(TargetArch::AVX); + if (Cpu::CpuFlagsCache::have_AVX2) + result |= static_cast(TargetArch::AVX2); + if (Cpu::CpuFlagsCache::have_AVX512F) + result |= static_cast(TargetArch::AVX512F); + return result; +} + +bool isArchSupported(TargetArch arch) +{ + static UInt32 arches = getSupportedArchs(); + return arch == TargetArch::Default || (arches & static_cast(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(); +} + +} diff --git a/src/Functions/TargetSpecific.h b/src/Functions/TargetSpecific.h new file mode 100644 index 00000000000..df7473686fd --- /dev/null +++ b/src/Functions/TargetSpecific.h @@ -0,0 +1,213 @@ +#pragma once + +#include + +/* This file contains macros and helpers for writing platform-dependent code. + * + * Macros DECLARE__SPECIFIC_CODE will wrap code inside it into the + * namespace TargetSpecific:: 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::::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 + +} diff --git a/src/Functions/VectorExtension.h b/src/Functions/VectorExtension.h new file mode 100644 index 00000000000..24c2ae9a18f --- /dev/null +++ b/src/Functions/VectorExtension.h @@ -0,0 +1,78 @@ +#pragma once + +#include +// 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 + 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 +using UInt8x = typename detail::DummyStruct::UInt8Type; +template +using UInt16x = typename detail::DummyStruct::UInt16Type; +template +using UInt32x = typename detail::DummyStruct::UInt32Type; +template +using UInt64x = typename detail::DummyStruct::UInt64Type; + +} diff --git a/src/Functions/generateUUIDv4.cpp b/src/Functions/generateUUIDv4.cpp index 39013519d2f..7dbb73c0cf3 100644 --- a/src/Functions/generateUUIDv4.cpp +++ b/src/Functions/generateUUIDv4.cpp @@ -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(); } 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(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(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation(); + #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(context); + } + +private: + ImplementationSelector selector; +}; + void registerFunctionGenerateUUIDv4(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index 238499f8def..bff92d7738d 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include @@ -153,6 +155,12 @@ enum class Method WGS84_METERS, }; +} + +DECLARE_MULTITARGET_CODE( + +namespace +{ template 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 class FunctionGeoDistance : public IFunction { @@ -230,8 +237,6 @@ public: : ((method == Method::SPHERE_METERS) ? "greatCircleDistance" : "geoDistance"); - static FunctionPtr create(const Context &) { return std::make_shared>(); } - private: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 4; } @@ -272,6 +277,40 @@ private: } }; +) // DECLARE_MULTITARGET_CODE + +template +class FunctionGeoDistance : public TargetSpecific::Default::FunctionGeoDistance +{ +public: + explicit FunctionGeoDistance(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); + #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>(context); + } + +private: + ImplementationSelector selector; +}; void registerFunctionGeoDistance(FunctionFactory & factory) { diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index bad4b199ee2..ebf2f752b66 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -99,7 +99,8 @@ public: argument_types.emplace_back(arguments.back().type); typename ColumnVector::Container vec_to(1); - RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); + + TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); ToType value = vec_to[0]; return std::make_unique>(value, argument_types, return_type); diff --git a/src/Functions/randomFixedString.cpp b/src/Functions/randomFixedString.cpp index 9fb7550346b..669dc084999 100644 --- a/src/Functions/randomFixedString.cpp +++ b/src/Functions/randomFixedString.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -21,13 +23,12 @@ namespace ErrorCodes /* Generate random fixed string with fully random bytes (including zero). */ -class FunctionRandomFixedString : public IFunction +template +class FunctionRandomFixedStringImpl : public IFunction { public: static constexpr auto name = "randomFixedString"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - 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(pos, rng()); - pos += sizeof(UInt64); // We have padding in column buffers that we can overwrite. - } + RandImpl::execute(reinterpret_cast(data_to.data()), total_size); block.getByPosition(result).column = std::move(col_to); } }; +class FunctionRandomFixedString : public FunctionRandomFixedStringImpl +{ +public: + explicit FunctionRandomFixedString(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + #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(context); + } + +private: + ImplementationSelector selector; +}; + void registerFunctionRandomFixedString(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/randomString.cpp b/src/Functions/randomString.cpp index 4ea470f0a65..df3278c3800 100644 --- a/src/Functions/randomString.cpp +++ b/src/Functions/randomString.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -19,13 +21,12 @@ namespace ErrorCodes /* Generate random string of specified length with fully random bytes (including zero). */ -class FunctionRandomString : public IFunction +template +class FunctionRandomStringImpl : public IFunction { public: static constexpr auto name = "randomString"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - 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(pos, rng()); - pos += sizeof(UInt64); // We have padding in column buffers that we can overwrite. - } + RandImpl::execute(reinterpret_cast(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 +{ +public: + explicit FunctionRandomString(const Context & context) : selector(context) + { + selector.registerImplementation>(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation>(); + #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(context); + } + +private: + ImplementationSelector selector; +}; + void registerFunctionRandomString(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index cd97dfe6bc3..17940389ad1 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -365,6 +365,7 @@ SRCS( subtractYears.cpp tan.cpp tanh.cpp + TargetSpecific.cpp tgamma.cpp throwIf.cpp timeSlot.cpp diff --git a/tests/performance/rand.xml b/tests/performance/rand.xml new file mode 100644 index 00000000000..ed629e5a2a7 --- /dev/null +++ b/tests/performance/rand.xml @@ -0,0 +1,24 @@ + + + + 10000 + + + 20000 + + + + + + + table + + zeros(100000000) + + + + + SELECT count() FROM (SELECT rand() FROM {table}) + SELECT count() FROM (SELECT rand64() FROM {table}) + SELECT count() FROM (SELECT generateUUIDv4() FROM {table}) +