mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 11:02:08 +00:00
Merge remote-tracking branch 'upstream/master' into fix25
This commit is contained in:
commit
6c89980158
5
.gitmodules
vendored
5
.gitmodules
vendored
@ -93,10 +93,13 @@
|
|||||||
url = https://github.com/ClickHouse-Extras/libunwind.git
|
url = https://github.com/ClickHouse-Extras/libunwind.git
|
||||||
[submodule "contrib/simdjson"]
|
[submodule "contrib/simdjson"]
|
||||||
path = contrib/simdjson
|
path = contrib/simdjson
|
||||||
url = https://github.com/lemire/simdjson.git
|
url = https://github.com/ClickHouse-Extras/simdjson.git
|
||||||
[submodule "contrib/rapidjson"]
|
[submodule "contrib/rapidjson"]
|
||||||
path = contrib/rapidjson
|
path = contrib/rapidjson
|
||||||
url = https://github.com/Tencent/rapidjson
|
url = https://github.com/Tencent/rapidjson
|
||||||
[submodule "contrib/mimalloc"]
|
[submodule "contrib/mimalloc"]
|
||||||
path = contrib/mimalloc
|
path = contrib/mimalloc
|
||||||
url = https://github.com/ClickHouse-Extras/mimalloc
|
url = https://github.com/ClickHouse-Extras/mimalloc
|
||||||
|
[submodule "contrib/fastops"]
|
||||||
|
path = contrib/fastops
|
||||||
|
url = https://github.com/ClickHouse-Extras/fastops
|
||||||
|
@ -476,6 +476,7 @@ include (cmake/find_hyperscan.cmake)
|
|||||||
include (cmake/find_mimalloc.cmake)
|
include (cmake/find_mimalloc.cmake)
|
||||||
include (cmake/find_simdjson.cmake)
|
include (cmake/find_simdjson.cmake)
|
||||||
include (cmake/find_rapidjson.cmake)
|
include (cmake/find_rapidjson.cmake)
|
||||||
|
include (cmake/find_fastops.cmake)
|
||||||
|
|
||||||
find_contrib_lib(cityhash)
|
find_contrib_lib(cityhash)
|
||||||
find_contrib_lib(farmhash)
|
find_contrib_lib(farmhash)
|
||||||
@ -568,4 +569,5 @@ if (GLIBC_COMPATIBILITY OR USE_INTERNAL_UNWIND_LIBRARY_FOR_EXCEPTION_HANDLING)
|
|||||||
add_default_dependencies(base64)
|
add_default_dependencies(base64)
|
||||||
add_default_dependencies(readpassphrase)
|
add_default_dependencies(readpassphrase)
|
||||||
add_default_dependencies(unwind_static)
|
add_default_dependencies(unwind_static)
|
||||||
|
add_default_dependencies(fastops)
|
||||||
endif ()
|
endif ()
|
||||||
|
15
cmake/find_fastops.cmake
Normal file
15
cmake/find_fastops.cmake
Normal file
@ -0,0 +1,15 @@
|
|||||||
|
option (ENABLE_FASTOPS "Enable fast vectorized mathematical functions library by Michael Parakhin" ${NOT_UNBUNDLED})
|
||||||
|
|
||||||
|
if (ENABLE_FASTOPS)
|
||||||
|
if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/fastops/fastops/fastops.h")
|
||||||
|
message(FATAL_ERROR "submodule contrib/fastops is missing. to fix try run: \n git submodule update --init --recursive")
|
||||||
|
set(USE_FASTOPS 0)
|
||||||
|
endif()
|
||||||
|
set (USE_FASTOPS 1)
|
||||||
|
set (FASTOPS_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/fastops/)
|
||||||
|
set (FASTOPS_LIBRARY fastops)
|
||||||
|
else ()
|
||||||
|
set(USE_FASTOPS 0)
|
||||||
|
endif ()
|
||||||
|
|
||||||
|
message (STATUS "Using fastops")
|
4
contrib/CMakeLists.txt
vendored
4
contrib/CMakeLists.txt
vendored
@ -330,3 +330,7 @@ endif()
|
|||||||
if (USE_MIMALLOC)
|
if (USE_MIMALLOC)
|
||||||
add_subdirectory (mimalloc)
|
add_subdirectory (mimalloc)
|
||||||
endif()
|
endif()
|
||||||
|
|
||||||
|
if (USE_FASTOPS)
|
||||||
|
add_subdirectory (fastops-cmake)
|
||||||
|
endif()
|
||||||
|
1
contrib/fastops
vendored
Submodule
1
contrib/fastops
vendored
Submodule
@ -0,0 +1 @@
|
|||||||
|
Subproject commit d2c85c5d6549cfd648a7f31ef7b14341881ff8ae
|
20
contrib/fastops-cmake/CMakeLists.txt
Normal file
20
contrib/fastops-cmake/CMakeLists.txt
Normal file
@ -0,0 +1,20 @@
|
|||||||
|
set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/fastops)
|
||||||
|
|
||||||
|
set(SRCS "")
|
||||||
|
|
||||||
|
if(HAVE_AVX)
|
||||||
|
set (SRCS ${SRCS} ${LIBRARY_DIR}/fastops/avx/ops_avx.cpp ${LIBRARY_DIR}/fastops/core/FastIntrinsics.cpp)
|
||||||
|
set_source_files_properties(${LIBRARY_DIR}/fastops/avx/ops_avx.cpp PROPERTIES COMPILE_FLAGS "-mavx -DNO_AVX2")
|
||||||
|
set_source_files_properties(${LIBRARY_DIR}/fastops/core/FastIntrinsics.cpp PROPERTIES COMPILE_FLAGS "-mavx -DNO_AVX2")
|
||||||
|
endif()
|
||||||
|
|
||||||
|
if(HAVE_AVX2)
|
||||||
|
set (SRCS ${SRCS} ${LIBRARY_DIR}/fastops/avx2/ops_avx2.cpp)
|
||||||
|
set_source_files_properties(${LIBRARY_DIR}/fastops/avx2/ops_avx2.cpp PROPERTIES COMPILE_FLAGS "-mavx2 -mfma")
|
||||||
|
endif()
|
||||||
|
|
||||||
|
set (SRCS ${SRCS} ${LIBRARY_DIR}/fastops/plain/ops_plain.cpp ${LIBRARY_DIR}/fastops/core/avx_id.cpp ${LIBRARY_DIR}/fastops/fastops.cpp)
|
||||||
|
|
||||||
|
add_library(fastops ${SRCS})
|
||||||
|
|
||||||
|
target_include_directories(fastops SYSTEM PUBLIC "${LIBRARY_DIR}")
|
2
contrib/simdjson
vendored
2
contrib/simdjson
vendored
@ -1 +1 @@
|
|||||||
Subproject commit 3bd3116cf8faf6d482dc31423b16533bfa2696f7
|
Subproject commit e3f6322af762213ff2087ce3366bf9541c7fd355
|
@ -10,7 +10,7 @@
|
|||||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||||
#include <Common/ThreadPool.h>
|
#include <Common/ThreadPool.h>
|
||||||
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
|
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
|
||||||
#include <Storages/StorageBlockInserted.h>
|
#include <Storages/StorageValues.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -183,7 +183,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n
|
|||||||
/// but it will contain single block (that is INSERT-ed into main table).
|
/// but it will contain single block (that is INSERT-ed into main table).
|
||||||
/// InterpreterSelectQuery will do processing of alias columns.
|
/// InterpreterSelectQuery will do processing of alias columns.
|
||||||
Context local_context = *views_context;
|
Context local_context = *views_context;
|
||||||
local_context.addViewSource(StorageBlockInserted::create(block, storage));
|
local_context.addViewSource(StorageValues::create(storage->getDatabaseName(), storage->getTableName(), storage->getColumns(), block));
|
||||||
InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions());
|
InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions());
|
||||||
|
|
||||||
BlockInputStreamPtr in = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
|
BlockInputStreamPtr in = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
|
||||||
|
@ -20,6 +20,7 @@ target_link_libraries(clickhouse_functions
|
|||||||
${METROHASH_LIBRARIES}
|
${METROHASH_LIBRARIES}
|
||||||
murmurhash
|
murmurhash
|
||||||
${BASE64_LIBRARY}
|
${BASE64_LIBRARY}
|
||||||
|
${FASTOPS_LIBRARY}
|
||||||
|
|
||||||
PRIVATE
|
PRIVATE
|
||||||
${ZLIB_LIBRARIES}
|
${ZLIB_LIBRARIES}
|
||||||
@ -31,7 +32,7 @@ if (OPENSSL_CRYPTO_LIBRARY)
|
|||||||
endif()
|
endif()
|
||||||
|
|
||||||
target_include_directories(clickhouse_functions PRIVATE ${CMAKE_CURRENT_BINARY_DIR}/include)
|
target_include_directories(clickhouse_functions PRIVATE ${CMAKE_CURRENT_BINARY_DIR}/include)
|
||||||
target_include_directories(clickhouse_functions SYSTEM BEFORE PUBLIC ${DIVIDE_INCLUDE_DIR} ${METROHASH_INCLUDE_DIR})
|
target_include_directories(clickhouse_functions SYSTEM PRIVATE ${DIVIDE_INCLUDE_DIR} ${METROHASH_INCLUDE_DIR})
|
||||||
|
|
||||||
if (CONSISTENT_HASHING_INCLUDE_DIR)
|
if (CONSISTENT_HASHING_INCLUDE_DIR)
|
||||||
target_include_directories (clickhouse_functions PRIVATE ${CONSISTENT_HASHING_INCLUDE_DIR})
|
target_include_directories (clickhouse_functions PRIVATE ${CONSISTENT_HASHING_INCLUDE_DIR})
|
||||||
@ -48,7 +49,11 @@ if (USE_ICU)
|
|||||||
endif ()
|
endif ()
|
||||||
|
|
||||||
if (USE_VECTORCLASS)
|
if (USE_VECTORCLASS)
|
||||||
target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${VECTORCLASS_INCLUDE_DIR})
|
target_include_directories (clickhouse_functions SYSTEM PRIVATE ${VECTORCLASS_INCLUDE_DIR})
|
||||||
|
endif ()
|
||||||
|
|
||||||
|
if (USE_FASTOPS)
|
||||||
|
target_include_directories (clickhouse_functions SYSTEM PRIVATE ${FASTOPS_INCLUDE_DIR})
|
||||||
endif ()
|
endif ()
|
||||||
|
|
||||||
if (ENABLE_TESTS)
|
if (ENABLE_TESTS)
|
||||||
|
@ -31,6 +31,14 @@
|
|||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
/** FastOps is a fast vector math library from Michael Parakhin (former Yandex CTO),
|
||||||
|
* Enabled by default.
|
||||||
|
*/
|
||||||
|
#if USE_FASTOPS
|
||||||
|
#include <fastops/fastops.h>
|
||||||
|
#endif
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -41,16 +49,14 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
|
|
||||||
template <typename Impl>
|
template <typename Impl>
|
||||||
class FunctionMathUnaryFloat64 : public IFunction
|
class FunctionMathUnary : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
static constexpr auto name = Impl::name;
|
static constexpr auto name = Impl::name;
|
||||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathUnaryFloat64>(); }
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathUnary>(); }
|
||||||
static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration");
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
|
|
||||||
size_t getNumberOfArguments() const override { return 1; }
|
size_t getNumberOfArguments() const override { return 1; }
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
@ -60,11 +66,35 @@ private:
|
|||||||
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
|
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
|
/// Integers are converted to Float64.
|
||||||
|
if (Impl::always_returns_float64 || !isFloat(arg))
|
||||||
return std::make_shared<DataTypeFloat64>();
|
return std::make_shared<DataTypeFloat64>();
|
||||||
|
else
|
||||||
|
return arg;
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <typename T, typename ReturnType>
|
||||||
static void executeInIterations(const T * src_data, Float64 * dst_data, size_t size)
|
static void executeInIterations(const T * src_data, ReturnType * dst_data, size_t size)
|
||||||
|
{
|
||||||
|
if constexpr (Impl::rows_per_iteration == 0)
|
||||||
|
{
|
||||||
|
/// Process all data as a whole and use FastOps implementation
|
||||||
|
|
||||||
|
/// If the argument is integer, convert to Float64 beforehand
|
||||||
|
if constexpr (!std::is_floating_point_v<T>)
|
||||||
|
{
|
||||||
|
PODArray<Float64> tmp_vec(size);
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
tmp_vec[i] = src_data[i];
|
||||||
|
|
||||||
|
Impl::execute(tmp_vec.data(), size, dst_data);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
Impl::execute(src_data, size, dst_data);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
{
|
{
|
||||||
const size_t rows_remaining = size % Impl::rows_per_iteration;
|
const size_t rows_remaining = size % Impl::rows_per_iteration;
|
||||||
const size_t rows_size = size - rows_remaining;
|
const size_t rows_size = size - rows_remaining;
|
||||||
@ -77,21 +107,22 @@ private:
|
|||||||
T src_remaining[Impl::rows_per_iteration];
|
T src_remaining[Impl::rows_per_iteration];
|
||||||
memcpy(src_remaining, &src_data[rows_size], rows_remaining * sizeof(T));
|
memcpy(src_remaining, &src_data[rows_size], rows_remaining * sizeof(T));
|
||||||
memset(src_remaining + rows_remaining, 0, (Impl::rows_per_iteration - rows_remaining) * sizeof(T));
|
memset(src_remaining + rows_remaining, 0, (Impl::rows_per_iteration - rows_remaining) * sizeof(T));
|
||||||
Float64 dst_remaining[Impl::rows_per_iteration];
|
ReturnType dst_remaining[Impl::rows_per_iteration];
|
||||||
|
|
||||||
Impl::execute(src_remaining, dst_remaining);
|
Impl::execute(src_remaining, dst_remaining);
|
||||||
|
|
||||||
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
|
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(ReturnType));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <typename T, typename ReturnType>
|
||||||
static bool execute(Block & block, const ColumnVector<T> * col, const size_t result)
|
static bool execute(Block & block, const ColumnVector<T> * col, const size_t result)
|
||||||
{
|
{
|
||||||
const auto & src_data = col->getData();
|
const auto & src_data = col->getData();
|
||||||
const size_t size = src_data.size();
|
const size_t size = src_data.size();
|
||||||
|
|
||||||
auto dst = ColumnVector<Float64>::create();
|
auto dst = ColumnVector<ReturnType>::create();
|
||||||
auto & dst_data = dst->getData();
|
auto & dst_data = dst->getData();
|
||||||
dst_data.resize(size);
|
dst_data.resize(size);
|
||||||
|
|
||||||
@ -101,19 +132,19 @@ private:
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <typename T, typename ReturnType>
|
||||||
static bool execute(Block & block, const ColumnDecimal<T> * col, const size_t result)
|
static bool execute(Block & block, const ColumnDecimal<T> * col, const size_t result)
|
||||||
{
|
{
|
||||||
const auto & src_data = col->getData();
|
const auto & src_data = col->getData();
|
||||||
const size_t size = src_data.size();
|
const size_t size = src_data.size();
|
||||||
UInt32 scale = src_data.getScale();
|
UInt32 scale = src_data.getScale();
|
||||||
|
|
||||||
auto dst = ColumnVector<Float64>::create();
|
auto dst = ColumnVector<ReturnType>::create();
|
||||||
auto & dst_data = dst->getData();
|
auto & dst_data = dst->getData();
|
||||||
dst_data.resize(size);
|
dst_data.resize(size);
|
||||||
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
for (size_t i = 0; i < size; ++i)
|
||||||
dst_data[i] = convertFromDecimal<DataTypeDecimal<T>, DataTypeNumber<Float64>>(src_data[i], scale);
|
dst_data[i] = convertFromDecimal<DataTypeDecimal<T>, DataTypeNumber<ReturnType>>(src_data[i], scale);
|
||||||
|
|
||||||
executeInIterations(dst_data.data(), dst_data.data(), size);
|
executeInIterations(dst_data.data(), dst_data.data(), size);
|
||||||
|
|
||||||
@ -131,10 +162,11 @@ private:
|
|||||||
{
|
{
|
||||||
using Types = std::decay_t<decltype(types)>;
|
using Types = std::decay_t<decltype(types)>;
|
||||||
using Type = typename Types::RightType;
|
using Type = typename Types::RightType;
|
||||||
|
using ReturnType = std::conditional_t<Impl::always_returns_float64 || !std::is_floating_point_v<Type>, Float64, Type>;
|
||||||
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
|
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
|
||||||
|
|
||||||
const auto col_vec = checkAndGetColumn<ColVecType>(col.column.get());
|
const auto col_vec = checkAndGetColumn<ColVecType>(col.column.get());
|
||||||
return execute<Type>(block, col_vec, result);
|
return execute<Type, ReturnType>(block, col_vec, result);
|
||||||
};
|
};
|
||||||
|
|
||||||
if (!callOnBasicType<void, true, true, true, false>(col.type->getTypeId(), call))
|
if (!callOnBasicType<void, true, true, true, false>(col.type->getTypeId(), call))
|
||||||
@ -149,6 +181,7 @@ struct UnaryFunctionPlain
|
|||||||
{
|
{
|
||||||
static constexpr auto name = Name::name;
|
static constexpr auto name = Name::name;
|
||||||
static constexpr auto rows_per_iteration = 1;
|
static constexpr auto rows_per_iteration = 1;
|
||||||
|
static constexpr bool always_returns_float64 = true;
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
static void execute(const T * src, Float64 * dst)
|
static void execute(const T * src, Float64 * dst)
|
||||||
@ -164,6 +197,7 @@ struct UnaryFunctionVectorized
|
|||||||
{
|
{
|
||||||
static constexpr auto name = Name::name;
|
static constexpr auto name = Name::name;
|
||||||
static constexpr auto rows_per_iteration = 2;
|
static constexpr auto rows_per_iteration = 2;
|
||||||
|
static constexpr bool always_returns_float64 = true;
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
static void execute(const T * src, Float64 * dst)
|
static void execute(const T * src, Float64 * dst)
|
@ -25,14 +25,14 @@ struct SimdJSONParser
|
|||||||
|
|
||||||
void preallocate(size_t max_size)
|
void preallocate(size_t max_size)
|
||||||
{
|
{
|
||||||
if (!pj.allocateCapacity(max_size))
|
if (!pj.allocate_capacity(max_size))
|
||||||
throw Exception{"Can not allocate memory for " + std::to_string(max_size) + " units when parsing JSON",
|
throw Exception{"Can not allocate memory for " + std::to_string(max_size) + " units when parsing JSON",
|
||||||
ErrorCodes::CANNOT_ALLOCATE_MEMORY};
|
ErrorCodes::CANNOT_ALLOCATE_MEMORY};
|
||||||
}
|
}
|
||||||
|
|
||||||
bool parse(const StringRef & json) { return !json_parse(json.data, json.size, pj); }
|
bool parse(const StringRef & json) { return !json_parse(json.data, json.size, pj); }
|
||||||
|
|
||||||
using Iterator = simdjson::ParsedJson::iterator;
|
using Iterator = simdjson::ParsedJson::Iterator;
|
||||||
Iterator getRoot() { return Iterator{pj}; }
|
Iterator getRoot() { return Iterator{pj}; }
|
||||||
|
|
||||||
static bool isInt64(const Iterator & it) { return it.is_integer(); }
|
static bool isInt64(const Iterator & it) { return it.is_integer(); }
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct AcosName { static constexpr auto name = "acos"; };
|
struct AcosName { static constexpr auto name = "acos"; };
|
||||||
using FunctionAcos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AcosName, acos>>;
|
using FunctionAcos = FunctionMathUnary<UnaryFunctionVectorized<AcosName, acos>>;
|
||||||
|
|
||||||
void registerFunctionAcos(FunctionFactory & factory)
|
void registerFunctionAcos(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct AsinName { static constexpr auto name = "asin"; };
|
struct AsinName { static constexpr auto name = "asin"; };
|
||||||
using FunctionAsin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AsinName, asin>>;
|
using FunctionAsin = FunctionMathUnary<UnaryFunctionVectorized<AsinName, asin>>;
|
||||||
|
|
||||||
void registerFunctionAsin(FunctionFactory & factory)
|
void registerFunctionAsin(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct AtanName { static constexpr auto name = "atan"; };
|
struct AtanName { static constexpr auto name = "atan"; };
|
||||||
using FunctionAtan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AtanName, atan>>;
|
using FunctionAtan = FunctionMathUnary<UnaryFunctionVectorized<AtanName, atan>>;
|
||||||
|
|
||||||
void registerFunctionAtan(FunctionFactory & factory)
|
void registerFunctionAtan(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct CbrtName { static constexpr auto name = "cbrt"; };
|
struct CbrtName { static constexpr auto name = "cbrt"; };
|
||||||
using FunctionCbrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CbrtName, cbrt>>;
|
using FunctionCbrt = FunctionMathUnary<UnaryFunctionVectorized<CbrtName, cbrt>>;
|
||||||
|
|
||||||
void registerFunctionCbrt(FunctionFactory & factory)
|
void registerFunctionCbrt(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -9,3 +9,4 @@
|
|||||||
#cmakedefine01 USE_SIMDJSON
|
#cmakedefine01 USE_SIMDJSON
|
||||||
#cmakedefine01 USE_RAPIDJSON
|
#cmakedefine01 USE_RAPIDJSON
|
||||||
#cmakedefine01 USE_H3
|
#cmakedefine01 USE_H3
|
||||||
|
#cmakedefine01 USE_FASTOPS
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct CosName { static constexpr auto name = "cos"; };
|
struct CosName { static constexpr auto name = "cos"; };
|
||||||
using FunctionCos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CosName, cos>>;
|
using FunctionCos = FunctionMathUnary<UnaryFunctionVectorized<CosName, cos>>;
|
||||||
|
|
||||||
void registerFunctionCos(FunctionFactory & factory)
|
void registerFunctionCos(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct ErfName { static constexpr auto name = "erf"; };
|
struct ErfName { static constexpr auto name = "erf"; };
|
||||||
using FunctionErf = FunctionMathUnaryFloat64<UnaryFunctionPlain<ErfName, std::erf>>;
|
using FunctionErf = FunctionMathUnary<UnaryFunctionPlain<ErfName, std::erf>>;
|
||||||
|
|
||||||
void registerFunctionErf(FunctionFactory & factory)
|
void registerFunctionErf(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct ErfcName { static constexpr auto name = "erfc"; };
|
struct ErfcName { static constexpr auto name = "erfc"; };
|
||||||
using FunctionErfc = FunctionMathUnaryFloat64<UnaryFunctionPlain<ErfcName, std::erfc>>;
|
using FunctionErfc = FunctionMathUnary<UnaryFunctionPlain<ErfcName, std::erfc>>;
|
||||||
|
|
||||||
void registerFunctionErfc(FunctionFactory & factory)
|
void registerFunctionErfc(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,34 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct ExpName { static constexpr auto name = "exp"; };
|
struct ExpName { static constexpr auto name = "exp"; };
|
||||||
using FunctionExp = FunctionMathUnaryFloat64<UnaryFunctionVectorized<ExpName, exp>>;
|
|
||||||
|
#if USE_FASTOPS
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
struct Impl
|
||||||
|
{
|
||||||
|
static constexpr auto name = ExpName::name;
|
||||||
|
static constexpr auto rows_per_iteration = 0;
|
||||||
|
static constexpr bool always_returns_float64 = false;
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
static void execute(const T * src, size_t size, T * dst)
|
||||||
|
{
|
||||||
|
NFastOps::Exp<true>(src, size, dst);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
using FunctionExp = FunctionMathUnary<Impl>;
|
||||||
|
|
||||||
|
#else
|
||||||
|
using FunctionExp = FunctionMathUnary<UnaryFunctionVectorized<ExpName, exp>>;
|
||||||
|
#endif
|
||||||
|
|
||||||
void registerFunctionExp(FunctionFactory & factory)
|
void registerFunctionExp(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,4 +1,4 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#if !USE_VECTORCLASS
|
#if !USE_VECTORCLASS
|
||||||
# include <common/preciseExp10.h>
|
# include <common/preciseExp10.h>
|
||||||
@ -9,7 +9,7 @@ namespace DB
|
|||||||
|
|
||||||
struct Exp10Name { static constexpr auto name = "exp10"; };
|
struct Exp10Name { static constexpr auto name = "exp10"; };
|
||||||
|
|
||||||
using FunctionExp10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp10Name,
|
using FunctionExp10 = FunctionMathUnary<UnaryFunctionVectorized<Exp10Name,
|
||||||
#if USE_VECTORCLASS
|
#if USE_VECTORCLASS
|
||||||
exp10
|
exp10
|
||||||
#else
|
#else
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct Exp2Name { static constexpr auto name = "exp2"; };
|
struct Exp2Name { static constexpr auto name = "exp2"; };
|
||||||
using FunctionExp2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp2Name, exp2>>;
|
using FunctionExp2 = FunctionMathUnary<UnaryFunctionVectorized<Exp2Name, exp2>>;
|
||||||
|
|
||||||
void registerFunctionExp2(FunctionFactory & factory)
|
void registerFunctionExp2(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct LGammaName { static constexpr auto name = "lgamma"; };
|
struct LGammaName { static constexpr auto name = "lgamma"; };
|
||||||
using FunctionLGamma = FunctionMathUnaryFloat64<UnaryFunctionPlain<LGammaName, std::lgamma>>;
|
using FunctionLGamma = FunctionMathUnary<UnaryFunctionPlain<LGammaName, std::lgamma>>;
|
||||||
|
|
||||||
void registerFunctionLGamma(FunctionFactory & factory)
|
void registerFunctionLGamma(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,34 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct LogName { static constexpr auto name = "log"; };
|
struct LogName { static constexpr auto name = "log"; };
|
||||||
using FunctionLog = FunctionMathUnaryFloat64<UnaryFunctionVectorized<LogName, log>>;
|
|
||||||
|
#if USE_FASTOPS
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
struct Impl
|
||||||
|
{
|
||||||
|
static constexpr auto name = LogName::name;
|
||||||
|
static constexpr auto rows_per_iteration = 0;
|
||||||
|
static constexpr bool always_returns_float64 = false;
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
static void execute(const T * src, size_t size, T * dst)
|
||||||
|
{
|
||||||
|
NFastOps::Log<true>(src, size, dst);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
using FunctionLog = FunctionMathUnary<Impl>;
|
||||||
|
|
||||||
|
#else
|
||||||
|
using FunctionLog = FunctionMathUnary<UnaryFunctionVectorized<LogName, log>>;
|
||||||
|
#endif
|
||||||
|
|
||||||
void registerFunctionLog(FunctionFactory & factory)
|
void registerFunctionLog(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct Log10Name { static constexpr auto name = "log10"; };
|
struct Log10Name { static constexpr auto name = "log10"; };
|
||||||
using FunctionLog10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log10Name, log10>>;
|
using FunctionLog10 = FunctionMathUnary<UnaryFunctionVectorized<Log10Name, log10>>;
|
||||||
|
|
||||||
void registerFunctionLog10(FunctionFactory & factory)
|
void registerFunctionLog10(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct Log2Name { static constexpr auto name = "log2"; };
|
struct Log2Name { static constexpr auto name = "log2"; };
|
||||||
using FunctionLog2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log2Name, log2>>;
|
using FunctionLog2 = FunctionMathUnary<UnaryFunctionVectorized<Log2Name, log2>>;
|
||||||
|
|
||||||
void registerFunctionLog2(FunctionFactory & factory)
|
void registerFunctionLog2(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -23,6 +23,8 @@ void registerFunctionTan(FunctionFactory & factory);
|
|||||||
void registerFunctionAsin(FunctionFactory & factory);
|
void registerFunctionAsin(FunctionFactory & factory);
|
||||||
void registerFunctionAcos(FunctionFactory & factory);
|
void registerFunctionAcos(FunctionFactory & factory);
|
||||||
void registerFunctionAtan(FunctionFactory & factory);
|
void registerFunctionAtan(FunctionFactory & factory);
|
||||||
|
void registerFunctionSigmoid(FunctionFactory & factory);
|
||||||
|
void registerFunctionTanh(FunctionFactory & factory);
|
||||||
void registerFunctionPow(FunctionFactory & factory);
|
void registerFunctionPow(FunctionFactory & factory);
|
||||||
|
|
||||||
void registerFunctionsMath(FunctionFactory & factory)
|
void registerFunctionsMath(FunctionFactory & factory)
|
||||||
@ -47,6 +49,8 @@ void registerFunctionsMath(FunctionFactory & factory)
|
|||||||
registerFunctionAsin(factory);
|
registerFunctionAsin(factory);
|
||||||
registerFunctionAcos(factory);
|
registerFunctionAcos(factory);
|
||||||
registerFunctionAtan(factory);
|
registerFunctionAtan(factory);
|
||||||
|
registerFunctionSigmoid(factory);
|
||||||
|
registerFunctionTanh(factory);
|
||||||
registerFunctionPow(factory);
|
registerFunctionPow(factory);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
46
dbms/src/Functions/sigmoid.cpp
Normal file
46
dbms/src/Functions/sigmoid.cpp
Normal file
@ -0,0 +1,46 @@
|
|||||||
|
#include <Functions/FunctionMathUnary.h>
|
||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct SigmoidName { static constexpr auto name = "sigmoid"; };
|
||||||
|
|
||||||
|
#if USE_FASTOPS
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
struct Impl
|
||||||
|
{
|
||||||
|
static constexpr auto name = SigmoidName::name;
|
||||||
|
static constexpr auto rows_per_iteration = 0;
|
||||||
|
static constexpr bool always_returns_float64 = false;
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
static void execute(const T * src, size_t size, T * dst)
|
||||||
|
{
|
||||||
|
NFastOps::Sigmoid<>(src, size, dst);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
using FunctionSigmoid = FunctionMathUnary<Impl>;
|
||||||
|
|
||||||
|
#else
|
||||||
|
|
||||||
|
static double sigmoid(double x)
|
||||||
|
{
|
||||||
|
return 1.0 / (1.0 + exp(-x));
|
||||||
|
}
|
||||||
|
|
||||||
|
using FunctionSigmoid = FunctionMathUnary<UnaryFunctionVectorized<SigmoidName, sigmoid>>;
|
||||||
|
|
||||||
|
#endif
|
||||||
|
|
||||||
|
void registerFunctionSigmoid(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionSigmoid>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct SinName { static constexpr auto name = "sin"; };
|
struct SinName { static constexpr auto name = "sin"; };
|
||||||
using FunctionSin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SinName, sin>>;
|
using FunctionSin = FunctionMathUnary<UnaryFunctionVectorized<SinName, sin>>;
|
||||||
|
|
||||||
void registerFunctionSin(FunctionFactory & factory)
|
void registerFunctionSin(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct SqrtName { static constexpr auto name = "sqrt"; };
|
struct SqrtName { static constexpr auto name = "sqrt"; };
|
||||||
using FunctionSqrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SqrtName, sqrt>>;
|
using FunctionSqrt = FunctionMathUnary<UnaryFunctionVectorized<SqrtName, sqrt>>;
|
||||||
|
|
||||||
void registerFunctionSqrt(FunctionFactory & factory)
|
void registerFunctionSqrt(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct TanName { static constexpr auto name = "tan"; };
|
struct TanName { static constexpr auto name = "tan"; };
|
||||||
using FunctionTan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<TanName, tan>>;
|
using FunctionTan = FunctionMathUnary<UnaryFunctionVectorized<TanName, tan>>;
|
||||||
|
|
||||||
void registerFunctionTan(FunctionFactory & factory)
|
void registerFunctionTan(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
46
dbms/src/Functions/tanh.cpp
Normal file
46
dbms/src/Functions/tanh.cpp
Normal file
@ -0,0 +1,46 @@
|
|||||||
|
#include <Functions/FunctionMathUnary.h>
|
||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct TanhName { static constexpr auto name = "tanh"; };
|
||||||
|
|
||||||
|
#if USE_FASTOPS
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
struct Impl
|
||||||
|
{
|
||||||
|
static constexpr auto name = TanhName::name;
|
||||||
|
static constexpr auto rows_per_iteration = 0;
|
||||||
|
static constexpr bool always_returns_float64 = false;
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
static void execute(const T * src, size_t size, T * dst)
|
||||||
|
{
|
||||||
|
NFastOps::Tanh<>(src, size, dst);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
using FunctionTanh = FunctionMathUnary<Impl>;
|
||||||
|
|
||||||
|
#else
|
||||||
|
|
||||||
|
static double tanh(double x)
|
||||||
|
{
|
||||||
|
return 2 / (1.0 + exp(-2 * x)) - 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
using FunctionTanh = FunctionMathUnary<UnaryFunctionVectorized<TanhName, tanh>>;
|
||||||
|
#endif
|
||||||
|
|
||||||
|
void registerFunctionTanh(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionTanh>(FunctionFactory::CaseInsensitive);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
@ -1,11 +1,11 @@
|
|||||||
#include <Functions/FunctionMathUnaryFloat64.h>
|
#include <Functions/FunctionMathUnary.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct TGammaName { static constexpr auto name = "tgamma"; };
|
struct TGammaName { static constexpr auto name = "tgamma"; };
|
||||||
using FunctionTGamma = FunctionMathUnaryFloat64<UnaryFunctionPlain<TGammaName, std::tgamma>>;
|
using FunctionTGamma = FunctionMathUnary<UnaryFunctionPlain<TGammaName, std::tgamma>>;
|
||||||
|
|
||||||
void registerFunctionTGamma(FunctionFactory & factory)
|
void registerFunctionTGamma(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
|
@ -133,7 +133,7 @@ private:
|
|||||||
String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification.
|
String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification.
|
||||||
/// Thus, used in HTTP interface. If not specified - then some globally default format is used.
|
/// Thus, used in HTTP interface. If not specified - then some globally default format is used.
|
||||||
TableAndCreateASTs external_tables; /// Temporary tables.
|
TableAndCreateASTs external_tables; /// Temporary tables.
|
||||||
StoragePtr view_source; /// Temporary StorageBlockInserted used to generate alias columns for materialized views
|
StoragePtr view_source; /// Temporary StorageValues used to generate alias columns for materialized views
|
||||||
Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id.
|
Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id.
|
||||||
Context * query_context = nullptr;
|
Context * query_context = nullptr;
|
||||||
Context * session_context = nullptr; /// Session context or nullptr. Could be equal to this.
|
Context * session_context = nullptr; /// Session context or nullptr. Could be equal to this.
|
||||||
|
@ -48,9 +48,7 @@
|
|||||||
#include <Storages/MergeTree/MergeTreeData.h>
|
#include <Storages/MergeTree/MergeTreeData.h>
|
||||||
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
|
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
|
||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
#include <Storages/StorageBlockInserted.h>
|
#include <Storages/StorageValues.h>
|
||||||
#include <Storages/StorageMergeTree.h>
|
|
||||||
#include <Storages/StorageReplicatedMergeTree.h>
|
|
||||||
|
|
||||||
#include <TableFunctions/ITableFunction.h>
|
#include <TableFunctions/ITableFunction.h>
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
@ -274,8 +272,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
|||||||
|
|
||||||
if (auto view_source = context.getViewSource())
|
if (auto view_source = context.getViewSource())
|
||||||
{
|
{
|
||||||
auto & storage_block = static_cast<const StorageBlockInserted &>(*view_source);
|
auto & storage_values = static_cast<const StorageValues &>(*view_source);
|
||||||
if (storage_block.getDatabaseName() == database_name && storage_block.getTableName() == table_name)
|
if (storage_values.getDatabaseName() == database_name && storage_values.getTableName() == table_name)
|
||||||
{
|
{
|
||||||
/// Read from view source.
|
/// Read from view source.
|
||||||
storage = context.getViewSource();
|
storage = context.getViewSource();
|
||||||
|
@ -1,59 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <ext/shared_ptr_helper.h>
|
|
||||||
|
|
||||||
#include <Core/NamesAndTypes.h>
|
|
||||||
#include <DataStreams/OneBlockInputStream.h>
|
|
||||||
#include <Storages/IStorage.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
/// The table has all the properties of another storage,
|
|
||||||
/// but will read single prepared block instead.
|
|
||||||
/// Used in PushingToViewsBlockOutputStream for generating alias columns
|
|
||||||
/// NOTE: Some of the properties seems redundant.
|
|
||||||
class StorageBlock : public ext::shared_ptr_helper<StorageBlock>, public IStorage
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
std::string getName() const override { return storage->getName(); }
|
|
||||||
std::string getTableName() const override { return storage->getTableName(); }
|
|
||||||
std::string getDatabaseName() const override { return storage->getDatabaseName(); }
|
|
||||||
bool isRemote() const override { return storage->isRemote(); }
|
|
||||||
bool supportsSampling() const override { return storage->supportsSampling(); }
|
|
||||||
bool supportsFinal() const override { return storage->supportsFinal(); }
|
|
||||||
bool supportsPrewhere() const override { return storage->supportsPrewhere(); }
|
|
||||||
bool supportsReplication() const override { return storage->supportsReplication(); }
|
|
||||||
bool supportsDeduplication() const override { return storage->supportsDeduplication(); }
|
|
||||||
bool supportsIndexForIn() const override { return storage->supportsIndexForIn(); }
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override
|
|
||||||
{
|
|
||||||
return storage->mayBenefitFromIndexForIn(left_in_operand, query_context);
|
|
||||||
}
|
|
||||||
ASTPtr getPartitionKeyAST() const override { return storage->getPartitionKeyAST(); }
|
|
||||||
ASTPtr getSortingKeyAST() const override { return storage->getSortingKeyAST(); }
|
|
||||||
ASTPtr getPrimaryKeyAST() const override { return storage->getPrimaryKeyAST(); }
|
|
||||||
ASTPtr getSamplingKeyAST() const override { return storage->getSamplingKeyAST(); }
|
|
||||||
Names getColumnsRequiredForPartitionKey() const override { return storage->getColumnsRequiredForPartitionKey(); }
|
|
||||||
Names getColumnsRequiredForSortingKey() const override { return storage->getColumnsRequiredForSortingKey(); }
|
|
||||||
Names getColumnsRequiredForPrimaryKey() const override { return storage->getColumnsRequiredForPrimaryKey(); }
|
|
||||||
Names getColumnsRequiredForSampling() const override { return storage->getColumnsRequiredForSampling(); }
|
|
||||||
Names getColumnsRequiredForFinal() const override { return storage->getColumnsRequiredForFinal(); }
|
|
||||||
|
|
||||||
BlockInputStreams read(const Names &, const SelectQueryInfo &, const Context &, QueryProcessingStage::Enum, size_t, unsigned) override
|
|
||||||
{
|
|
||||||
return {std::make_shared<OneBlockInputStream>(std::move(block))};
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
|
||||||
Block block;
|
|
||||||
StoragePtr storage;
|
|
||||||
|
|
||||||
protected:
|
|
||||||
StorageBlock(Block block_, StoragePtr storage_)
|
|
||||||
: IStorage{storage_->getColumns()}, block(std::move(block_)), storage(storage_)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -1,63 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <ext/shared_ptr_helper.h>
|
|
||||||
|
|
||||||
#include <Core/NamesAndTypes.h>
|
|
||||||
#include <DataStreams/OneBlockInputStream.h>
|
|
||||||
#include <Storages/IStorage.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
/// Allows to look at the Block that was inserted or is going to be inserted
|
|
||||||
/// to a table as if we read this block from that table.
|
|
||||||
|
|
||||||
/// The table has all the properties of another storage,
|
|
||||||
/// but will read single prepared block instead.
|
|
||||||
/// Used in PushingToViewsBlockOutputStream for generating alias columns
|
|
||||||
/// NOTE: Some of the properties seems redundant.
|
|
||||||
class StorageBlockInserted : public ext::shared_ptr_helper<StorageBlockInserted>, public IStorage
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
std::string getName() const override { return storage->getName(); }
|
|
||||||
std::string getTableName() const override { return storage->getTableName(); }
|
|
||||||
std::string getDatabaseName() const override { return storage->getDatabaseName(); }
|
|
||||||
bool isRemote() const override { return storage->isRemote(); }
|
|
||||||
bool supportsSampling() const override { return storage->supportsSampling(); }
|
|
||||||
bool supportsFinal() const override { return storage->supportsFinal(); }
|
|
||||||
bool supportsPrewhere() const override { return storage->supportsPrewhere(); }
|
|
||||||
bool supportsReplication() const override { return storage->supportsReplication(); }
|
|
||||||
bool supportsDeduplication() const override { return storage->supportsDeduplication(); }
|
|
||||||
bool supportsIndexForIn() const override { return storage->supportsIndexForIn(); }
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override
|
|
||||||
{
|
|
||||||
return storage->mayBenefitFromIndexForIn(left_in_operand, query_context);
|
|
||||||
}
|
|
||||||
ASTPtr getPartitionKeyAST() const override { return storage->getPartitionKeyAST(); }
|
|
||||||
ASTPtr getSortingKeyAST() const override { return storage->getSortingKeyAST(); }
|
|
||||||
ASTPtr getPrimaryKeyAST() const override { return storage->getPrimaryKeyAST(); }
|
|
||||||
ASTPtr getSamplingKeyAST() const override { return storage->getSamplingKeyAST(); }
|
|
||||||
Names getColumnsRequiredForPartitionKey() const override { return storage->getColumnsRequiredForPartitionKey(); }
|
|
||||||
Names getColumnsRequiredForSortingKey() const override { return storage->getColumnsRequiredForSortingKey(); }
|
|
||||||
Names getColumnsRequiredForPrimaryKey() const override { return storage->getColumnsRequiredForPrimaryKey(); }
|
|
||||||
Names getColumnsRequiredForSampling() const override { return storage->getColumnsRequiredForSampling(); }
|
|
||||||
Names getColumnsRequiredForFinal() const override { return storage->getColumnsRequiredForFinal(); }
|
|
||||||
|
|
||||||
BlockInputStreams read(const Names &, const SelectQueryInfo &, const Context &, QueryProcessingStage::Enum, size_t, unsigned) override
|
|
||||||
{
|
|
||||||
return {std::make_shared<OneBlockInputStream>(std::move(block))};
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
|
||||||
Block block;
|
|
||||||
StoragePtr storage;
|
|
||||||
|
|
||||||
protected:
|
|
||||||
StorageBlockInserted(Block block_, StoragePtr storage_)
|
|
||||||
: IStorage{storage_->getColumns()}, block(std::move(block_)), storage(storage_)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
46
dbms/tests/performance/math.xml
Normal file
46
dbms/tests/performance/math.xml
Normal file
@ -0,0 +1,46 @@
|
|||||||
|
<test>
|
||||||
|
<type>once</type>
|
||||||
|
|
||||||
|
<stop_conditions>
|
||||||
|
<any_of>
|
||||||
|
<average_speed_not_changing_for_ms>1000</average_speed_not_changing_for_ms>
|
||||||
|
<total_time_ms>10000</total_time_ms>
|
||||||
|
</any_of>
|
||||||
|
</stop_conditions>
|
||||||
|
|
||||||
|
<main_metric>
|
||||||
|
<max_rows_per_second />
|
||||||
|
</main_metric>
|
||||||
|
|
||||||
|
<substitutions>
|
||||||
|
<substitution>
|
||||||
|
<name>func</name>
|
||||||
|
<values>
|
||||||
|
<value>exp</value>
|
||||||
|
<value>log</value>
|
||||||
|
<value>exp2</value>
|
||||||
|
<value>log2</value>
|
||||||
|
<value>exp10</value>
|
||||||
|
<value>log10</value>
|
||||||
|
<value>sqrt</value>
|
||||||
|
<value>cbrt</value>
|
||||||
|
<value>erf</value>
|
||||||
|
<value>erfc</value>
|
||||||
|
<value>lgamma</value>
|
||||||
|
<value>tgamma</value>
|
||||||
|
<value>sin</value>
|
||||||
|
<value>cos</value>
|
||||||
|
<value>tan</value>
|
||||||
|
<value>asin</value>
|
||||||
|
<value>acos</value>
|
||||||
|
<value>atan</value>
|
||||||
|
<value>sigmoid</value>
|
||||||
|
<value>tanh</value>
|
||||||
|
</values>
|
||||||
|
</substitution>
|
||||||
|
</substitutions>
|
||||||
|
|
||||||
|
<query>SELECT count() FROM system.numbers WHERE NOT ignore({func}(toFloat64(number)))</query>
|
||||||
|
<query>SELECT count() FROM system.numbers WHERE NOT ignore({func}(toFloat32(number)))</query>
|
||||||
|
<query>SELECT count() FROM system.numbers WHERE NOT ignore({func}(number))</query>
|
||||||
|
</test>
|
@ -32,16 +32,16 @@ select tgamma(2) = 1;
|
|||||||
select tgamma(3) = 2;
|
select tgamma(3) = 2;
|
||||||
select tgamma(4) = 6;
|
select tgamma(4) = 6;
|
||||||
|
|
||||||
select sum(abs(lgamma(x + 1) - log(tgamma(x + 1))) < 1.0e-9) / count() from system.one array join range(10) as x;
|
select sum(abs(lgamma(x + 1) - log(tgamma(x + 1))) < 1.0e-8) / count() from system.one array join range(10) as x;
|
||||||
|
|
||||||
select abs(e() - arraySum(arrayMap(x -> 1 / tgamma(x + 1), range(13)))) < 1.0e-9;
|
select abs(e() - arraySum(arrayMap(x -> 1 / tgamma(x + 1), range(13)))) < 1.0e-9;
|
||||||
|
|
||||||
select log(0) = -inf;
|
select log(0) = -inf;
|
||||||
select log(1) = 0;
|
select log(1) = 0;
|
||||||
select log(e()) = 1;
|
select abs(log(e()) - 1) < 1e-8;
|
||||||
select log(exp(1)) = 1;
|
select abs(log(exp(1)) - 1) < 1e-8;
|
||||||
select log(exp(2)) = 2;
|
select abs(log(exp(2)) - 2) < 1e-8;
|
||||||
select sum(abs(log(exp(x)) - x) < 1.0e-9) / count() from system.one array join range(100) as x;
|
select sum(abs(log(exp(x)) - x) < 1e-8) / count() from system.one array join range(100) as x;
|
||||||
|
|
||||||
select exp2(-1) = 1/2;
|
select exp2(-1) = 1/2;
|
||||||
select exp2(0) = 1;
|
select exp2(0) = 1;
|
||||||
|
@ -1,70 +1,70 @@
|
|||||||
1 1.00 B 1 1.00 B 1 1.00 B
|
1.00 B 1.00 B 1.00 B
|
||||||
2.718282 2.72 B 2 2.00 B 2 2.00 B
|
2.72 B 2.00 B 2.00 B
|
||||||
7.389056 7.39 B 7 7.00 B 7 7.00 B
|
7.39 B 7.00 B 7.00 B
|
||||||
20.085537 20.09 B 20 20.00 B 20 20.00 B
|
20.09 B 20.00 B 20.00 B
|
||||||
54.59815 54.60 B 54 54.00 B 54 54.00 B
|
54.60 B 54.00 B 54.00 B
|
||||||
148.413159 148.41 B 148 148.00 B 148 148.00 B
|
148.41 B 148.00 B 148.00 B
|
||||||
403.428793 403.43 B 403 403.00 B 403 403.00 B
|
403.43 B 403.00 B 403.00 B
|
||||||
1096.633158 1.07 KiB 1096 1.07 KiB 1096 1.07 KiB
|
1.07 KiB 1.07 KiB 1.07 KiB
|
||||||
2980.957987 2.91 KiB 2980 2.91 KiB 2980 2.91 KiB
|
2.91 KiB 2.91 KiB 2.91 KiB
|
||||||
8103.083928 7.91 KiB 8103 7.91 KiB 8103 7.91 KiB
|
7.91 KiB 7.91 KiB 7.91 KiB
|
||||||
22026.465795 21.51 KiB 22026 21.51 KiB 22026 21.51 KiB
|
21.51 KiB 21.51 KiB 21.51 KiB
|
||||||
59874.141715 58.47 KiB 59874 58.47 KiB 59874 58.47 KiB
|
58.47 KiB 58.47 KiB 58.47 KiB
|
||||||
162754.791419 158.94 KiB 162754 158.94 KiB 162754 158.94 KiB
|
158.94 KiB 158.94 KiB 158.94 KiB
|
||||||
442413.392009 432.04 KiB 442413 432.04 KiB 442413 432.04 KiB
|
432.04 KiB 432.04 KiB 432.04 KiB
|
||||||
1202604.284165 1.15 MiB 1202604 1.15 MiB 1202604 1.15 MiB
|
1.15 MiB 1.15 MiB 1.15 MiB
|
||||||
3269017.372472 3.12 MiB 3269017 3.12 MiB 3269017 3.12 MiB
|
3.12 MiB 3.12 MiB 3.12 MiB
|
||||||
8886110.520508 8.47 MiB 8886110 8.47 MiB 8886110 8.47 MiB
|
8.47 MiB 8.47 MiB 8.47 MiB
|
||||||
24154952.753575 23.04 MiB 24154952 23.04 MiB 24154952 23.04 MiB
|
23.04 MiB 23.04 MiB 23.04 MiB
|
||||||
65659969.137331 62.62 MiB 65659969 62.62 MiB 65659969 62.62 MiB
|
62.62 MiB 62.62 MiB 62.62 MiB
|
||||||
178482300.963187 170.21 MiB 178482300 170.21 MiB 178482300 170.21 MiB
|
170.21 MiB 170.21 MiB 170.21 MiB
|
||||||
485165195.40979 462.69 MiB 485165195 462.69 MiB 485165195 462.69 MiB
|
462.69 MiB 462.69 MiB 462.69 MiB
|
||||||
1318815734.483214 1.23 GiB 1318815734 1.23 GiB 1318815734 1.23 GiB
|
1.23 GiB 1.23 GiB 1.23 GiB
|
||||||
3584912846.131592 3.34 GiB 3584912846 3.34 GiB -710054450 -677.16 MiB
|
3.34 GiB 3.34 GiB -2.00 GiB
|
||||||
9744803446.248903 9.08 GiB 9744803446 9.08 GiB 1154868854 1.08 GiB
|
9.08 GiB 9.08 GiB -2.00 GiB
|
||||||
26489122129.84347 24.67 GiB 26489122129 24.67 GiB 719318353 686.00 MiB
|
24.67 GiB 24.67 GiB -2.00 GiB
|
||||||
72004899337.38588 67.06 GiB 72004899337 67.06 GiB -1009544695 -962.78 MiB
|
67.06 GiB 67.06 GiB -2.00 GiB
|
||||||
195729609428.83878 182.29 GiB 195729609428 182.29 GiB -1838886188 -1.71 GiB
|
182.29 GiB 182.29 GiB -2.00 GiB
|
||||||
532048240601.79865 495.51 GiB 532048240601 495.51 GiB -527704103 -503.26 MiB
|
495.51 GiB 495.51 GiB -2.00 GiB
|
||||||
1446257064291.475 1.32 TiB 1446257064291 1.32 TiB -1146914461 -1.07 GiB
|
1.32 TiB 1.32 TiB -2.00 GiB
|
||||||
3931334297144.042 3.58 TiB 3931334297144 3.58 TiB 1439221304 1.34 GiB
|
3.58 TiB 3.58 TiB -2.00 GiB
|
||||||
10686474581524.463 9.72 TiB 10686474581524 9.72 TiB 595949076 568.34 MiB
|
9.72 TiB 9.72 TiB -2.00 GiB
|
||||||
29048849665247.426 26.42 TiB 29048849665247 26.42 TiB 1985842399 1.85 GiB
|
26.42 TiB 26.42 TiB -2.00 GiB
|
||||||
78962960182680.69 71.82 TiB 78962960182680 71.82 TiB -13554280 -12.93 MiB
|
71.82 TiB 71.82 TiB -2.00 GiB
|
||||||
214643579785916.06 195.22 TiB 214643579785916 195.22 TiB -1705798980 -1.59 GiB
|
195.22 TiB 195.22 TiB -2.00 GiB
|
||||||
583461742527454.9 530.66 TiB 583461742527454 530.66 TiB -974699554 -929.55 MiB
|
530.66 TiB 530.66 TiB -2.00 GiB
|
||||||
1586013452313430.8 1.41 PiB 1586013452313430 1.41 PiB -2005982378 -1.87 GiB
|
1.41 PiB 1.41 PiB -2.00 GiB
|
||||||
4311231547115195 3.83 PiB 4311231547115195 3.83 PiB -790034757 -753.44 MiB
|
3.83 PiB 3.83 PiB -2.00 GiB
|
||||||
11719142372802612 10.41 PiB 11719142372802612 10.41 PiB 1983119412 1.85 GiB
|
10.41 PiB 10.41 PiB -2.00 GiB
|
||||||
31855931757113756 28.29 PiB 31855931757113756 28.29 PiB 408891804 389.95 MiB
|
28.29 PiB 28.29 PiB -2.00 GiB
|
||||||
86593400423993740 76.91 PiB 86593400423993744 76.91 PiB 673862032 642.64 MiB
|
76.91 PiB 76.91 PiB -2.00 GiB
|
||||||
235385266837020000 209.06 PiB 235385266837020000 209.06 PiB 791567712 754.90 MiB
|
209.06 PiB 209.06 PiB -2.00 GiB
|
||||||
639843493530054900 568.30 PiB 639843493530054912 568.30 PiB 1874080000 1.75 GiB
|
568.30 PiB 568.30 PiB -2.00 GiB
|
||||||
1739274941520501000 1.51 EiB 1739274941520500992 1.51 EiB 538007808 513.08 MiB
|
1.51 EiB 1.51 EiB -2.00 GiB
|
||||||
4727839468229346000 4.10 EiB 4727839468229346304 4.10 EiB 2061616128 1.92 GiB
|
4.10 EiB 4.10 EiB -2.00 GiB
|
||||||
12851600114359308000 11.15 EiB 12851600114359308288 11.15 EiB -1681813504 -1.57 GiB
|
11.15 EiB 11.15 EiB -2.00 GiB
|
||||||
34934271057485095000 30.30 EiB 0 0.00 B 0 0.00 B
|
30.30 EiB 0.00 B -2.00 GiB
|
||||||
94961194206024480000 82.37 EiB 0 0.00 B 0 0.00 B
|
82.37 EiB 0.00 B -2.00 GiB
|
||||||
258131288619006750000 223.89 EiB 0 0.00 B 0 0.00 B
|
223.89 EiB 0.00 B -2.00 GiB
|
||||||
701673591209763100000 608.60 EiB 0 0.00 B 0 0.00 B
|
608.60 EiB 0.00 B -2.00 GiB
|
||||||
1.9073465724950998e21 1.62 ZiB 0 0.00 B 0 0.00 B
|
1.62 ZiB 0.00 B -2.00 GiB
|
||||||
5.184705528587072e21 4.39 ZiB 0 0.00 B 0 0.00 B
|
4.39 ZiB 0.00 B -2.00 GiB
|
||||||
1.4093490824269389e22 11.94 ZiB 0 0.00 B 0 0.00 B
|
11.94 ZiB 0.00 B -2.00 GiB
|
||||||
3.831008000716577e22 32.45 ZiB 0 0.00 B 0 0.00 B
|
32.45 ZiB 0.00 B -2.00 GiB
|
||||||
1.0413759433029089e23 88.21 ZiB 0 0.00 B 0 0.00 B
|
88.21 ZiB 0.00 B -2.00 GiB
|
||||||
2.830753303274694e23 239.77 ZiB 0 0.00 B 0 0.00 B
|
239.77 ZiB 0.00 B -2.00 GiB
|
||||||
7.694785265142018e23 651.77 ZiB 0 0.00 B 0 0.00 B
|
651.77 ZiB 0.00 B -2.00 GiB
|
||||||
2.091659496012996e24 1.73 YiB 0 0.00 B 0 0.00 B
|
1.73 YiB 0.00 B -2.00 GiB
|
||||||
5.685719999335932e24 4.70 YiB 0 0.00 B 0 0.00 B
|
4.70 YiB 0.00 B -2.00 GiB
|
||||||
1.545538935590104e25 12.78 YiB 0 0.00 B 0 0.00 B
|
12.78 YiB 0.00 B -2.00 GiB
|
||||||
4.2012104037905144e25 34.75 YiB 0 0.00 B 0 0.00 B
|
34.75 YiB 0.00 B -2.00 GiB
|
||||||
1.1420073898156842e26 94.46 YiB 0 0.00 B 0 0.00 B
|
94.46 YiB 0.00 B -2.00 GiB
|
||||||
3.10429793570192e26 256.78 YiB 0 0.00 B 0 0.00 B
|
256.78 YiB 0.00 B -2.00 GiB
|
||||||
8.438356668741454e26 698.00 YiB 0 0.00 B 0 0.00 B
|
698.00 YiB 0.00 B -2.00 GiB
|
||||||
2.29378315946961e27 1897.37 YiB 0 0.00 B 0 0.00 B
|
1897.37 YiB 0.00 B -2.00 GiB
|
||||||
6.235149080811617e27 5157.59 YiB 0 0.00 B 0 0.00 B
|
5157.59 YiB 0.00 B -2.00 GiB
|
||||||
1.6948892444103338e28 14019.80 YiB 0 0.00 B 0 0.00 B
|
14019.80 YiB 0.00 B -2.00 GiB
|
||||||
4.607186634331292e28 38109.75 YiB 0 0.00 B 0 0.00 B
|
38109.75 YiB 0.00 B -2.00 GiB
|
||||||
1.2523631708422137e29 103593.05 YiB 0 0.00 B 0 0.00 B
|
103593.05 YiB 0.00 B -2.00 GiB
|
||||||
3.404276049931741e29 281595.11 YiB 0 0.00 B 0 0.00 B
|
281595.11 YiB 0.00 B -2.00 GiB
|
||||||
9.253781725587787e29 765454.88 YiB 0 0.00 B 0 0.00 B
|
765454.88 YiB 0.00 B -2.00 GiB
|
||||||
|
@ -1 +1,4 @@
|
|||||||
SELECT round(exp(number), 6) AS x, formatReadableSize(x), toUInt64(x) AS y, formatReadableSize(y), toInt32(y) AS z, formatReadableSize(z) FROM system.numbers LIMIT 70;
|
WITH round(exp(number), 6) AS x, toUInt64(x) AS y, toInt32(x) AS z
|
||||||
|
SELECT formatReadableSize(x), formatReadableSize(y), formatReadableSize(z)
|
||||||
|
FROM system.numbers
|
||||||
|
LIMIT 70;
|
||||||
|
@ -1,2 +1,2 @@
|
|||||||
SELECT quantileTiming(number) FROM (SELECT * FROM system.numbers LIMIT 10000);
|
SELECT quantileTiming(number) FROM (SELECT * FROM system.numbers LIMIT 10000);
|
||||||
SELECT floor(log(1 + number) / log(1.5)) AS k, count() AS c, quantileTiming(number % 10000) AS q FROM (SELECT * FROM system.numbers LIMIT 1000000) GROUP BY k ORDER BY k;
|
SELECT floor(log2(1 + number) / log2(1.5)) AS k, count() AS c, quantileTiming(number % 10000) AS q FROM (SELECT * FROM system.numbers LIMIT 1000000) GROUP BY k ORDER BY k;
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
42.4200 3.7476 42.419153766068966
|
42.4200 3.7476 42.419154
|
||||||
42.4200 5.4066 42.41786197045111
|
42.4200 5.4066 42.417862
|
||||||
42.4200 1.6275 42.413098391048806
|
42.4200 1.6275 42.413098
|
||||||
42.4200 6.513 42.419169
|
42.4200 6.513 42.419169
|
||||||
42.4200 3.4875 42.417263671875
|
42.4200 3.4875 42.417263671875
|
||||||
1.00000 0.8427007929497149 0.15729920705028513
|
1.00000 0.8427007929497149 0.15729920705028513
|
||||||
@ -8,9 +8,9 @@
|
|||||||
0.00 0 1 0
|
0.00 0 1 0
|
||||||
3.14159265 0 -1 -0
|
3.14159265 0 -1 -0
|
||||||
1.00 1.5707963267948966 0 0.7853981633974483
|
1.00 1.5707963267948966 0 0.7853981633974483
|
||||||
42.4200 3.7476 42.419153766068966
|
42.4200 3.7476 42.419154
|
||||||
42.4200 5.4066 42.41786197045111
|
42.4200 5.4066 42.417862
|
||||||
42.4200 1.6275 42.413098391048806
|
42.4200 1.6275 42.413098
|
||||||
42.4200 6.513 42.419169
|
42.4200 6.513 42.419169
|
||||||
42.4200 3.4875 42.417263671875
|
42.4200 3.4875 42.417263671875
|
||||||
1.00000 0.8427007929497149 0.15729920705028513
|
1.00000 0.8427007929497149 0.15729920705028513
|
||||||
@ -18,9 +18,9 @@
|
|||||||
0.00 0 1 0
|
0.00 0 1 0
|
||||||
3.14159265358979328 0 -1 -0
|
3.14159265358979328 0 -1 -0
|
||||||
1.00 1.5707963267948966 0 0.7853981633974483
|
1.00 1.5707963267948966 0 0.7853981633974483
|
||||||
42.4200 3.7476 42.419153766068966
|
42.4200 3.7476 42.419154
|
||||||
42.4200 5.4066 42.41786197045111
|
42.4200 5.4066 42.417862
|
||||||
42.4200 1.6275 42.413098391048806
|
42.4200 1.6275 42.413098
|
||||||
42.4200 6.513 42.419169
|
42.4200 6.513 42.419169
|
||||||
42.4200 3.4875 42.417263671875
|
42.4200 3.4875 42.417263671875
|
||||||
1.00000 0.8427007929497149 0.15729920705028513
|
1.00000 0.8427007929497149 0.15729920705028513
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log(x), 4) AS y, exp(y);
|
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log(x), 4) AS y, round(exp(y), 6);
|
||||||
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log2(x), 4) AS y, exp2(y);
|
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log2(x), 4) AS y, round(exp2(y), 6);
|
||||||
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, exp10(y);
|
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, round(exp10(y), 6);
|
||||||
|
|
||||||
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(sqrt(x), 3) AS y, y * y;
|
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(sqrt(x), 3) AS y, y * y;
|
||||||
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(cbrt(x), 4) AS y, toDecimal64(y, 4) * y * y;
|
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(cbrt(x), 4) AS y, toDecimal64(y, 4) * y * y;
|
||||||
@ -12,9 +12,9 @@ SELECT toDecimal32(pi(), 8) AS x, round(sin(x), 8), round(cos(x), 8), round(tan(
|
|||||||
SELECT toDecimal32('1.0', 2) AS x, asin(x), acos(x), atan(x);
|
SELECT toDecimal32('1.0', 2) AS x, asin(x), acos(x), atan(x);
|
||||||
|
|
||||||
|
|
||||||
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(log(x), 4) AS y, exp(y);
|
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(log(x), 4) AS y, round(exp(y), 6);
|
||||||
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(log2(x), 4) AS y, exp2(y);
|
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(log2(x), 4) AS y, round(exp2(y), 6);
|
||||||
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, exp10(y);
|
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, round(exp10(y), 6);
|
||||||
|
|
||||||
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(sqrt(x), 3) AS y, y * y;
|
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(sqrt(x), 3) AS y, y * y;
|
||||||
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(cbrt(x), 4) AS y, toDecimal64(y, 4) * y * y;
|
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(cbrt(x), 4) AS y, toDecimal64(y, 4) * y * y;
|
||||||
@ -26,9 +26,9 @@ SELECT toDecimal64(pi(), 17) AS x, round(sin(x), 8), round(cos(x), 8), round(tan
|
|||||||
SELECT toDecimal64('1.0', 2) AS x, asin(x), acos(x), atan(x);
|
SELECT toDecimal64('1.0', 2) AS x, asin(x), acos(x), atan(x);
|
||||||
|
|
||||||
|
|
||||||
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(log(x), 4) AS y, exp(y);
|
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(log(x), 4) AS y, round(exp(y), 6);
|
||||||
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(log2(x), 4) AS y, exp2(y);
|
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(log2(x), 4) AS y, round(exp2(y), 6);
|
||||||
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, exp10(y);
|
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, round(exp10(y), 6);
|
||||||
|
|
||||||
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(sqrt(x), 3) AS y, y * y;
|
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(sqrt(x), 3) AS y, y * y;
|
||||||
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(cbrt(x), 4) AS y, toDecimal64(y, 4) * y * y;
|
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(cbrt(x), 4) AS y, toDecimal64(y, 4) * y * y;
|
||||||
|
@ -19,7 +19,7 @@
|
|||||||
0 1 0 0 0 0
|
0 1 0 0 0 0
|
||||||
0 1 0 0 0 0
|
0 1 0 0 0 0
|
||||||
nan nan nan nan nan nan
|
nan nan nan nan nan nan
|
||||||
nan nan nan nan nan nan nan nan nan
|
nan 0 nan nan nan nan nan nan nan
|
||||||
nan nan nan nan nan
|
nan nan nan nan nan
|
||||||
nan nan nan nan nan nan nan
|
nan nan nan nan nan nan nan
|
||||||
-1 1
|
-1 1
|
||||||
|
@ -22,7 +22,7 @@ SELECT -nan = toFloat64(0.0), -nan != toFloat64(0.0), -nan < toFloat64(0.0), -na
|
|||||||
|
|
||||||
--SELECT 1 % nan, nan % 1, pow(x, 1), pow(1, x); -- TODO
|
--SELECT 1 % nan, nan % 1, pow(x, 1), pow(1, x); -- TODO
|
||||||
SELECT 1 + nan, 1 - nan, nan - 1, 1 * nan, 1 / nan, nan / 1;
|
SELECT 1 + nan, 1 - nan, nan - 1, 1 * nan, 1 / nan, nan / 1;
|
||||||
SELECT nan AS x, exp(x), exp2(x), exp10(x), log(x), log2(x), log10(x), sqrt(x), cbrt(x);
|
SELECT nan AS x, isFinite(exp(x)) /* exp(nan) is allowed to return inf */, exp2(x), exp10(x), log(x), log2(x), log10(x), sqrt(x), cbrt(x);
|
||||||
SELECT nan AS x, erf(x), erfc(x), lgamma(x), tgamma(x);
|
SELECT nan AS x, erf(x), erfc(x), lgamma(x), tgamma(x);
|
||||||
SELECT nan AS x, sin(x), cos(x), tan(x), asin(x), acos(x), atan(x);
|
SELECT nan AS x, sin(x), cos(x), tan(x), asin(x), acos(x), atan(x);
|
||||||
|
|
||||||
|
@ -23,4 +23,4 @@ SELECT number % 10 AS k, boundingRatio(1000 + number, number * 1.5 - 111) FROM n
|
|||||||
SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(2);
|
SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(2);
|
||||||
SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(1);
|
SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(1);
|
||||||
SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(1) WHERE 0;
|
SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(1) WHERE 0;
|
||||||
SELECT boundingRatio(number, exp(number)) = e() - 1 FROM numbers(2);
|
SELECT boundingRatio(number, exp(number)) = exp(1) - 1 FROM numbers(2);
|
||||||
|
@ -1,10 +1,10 @@
|
|||||||
0 0 6 6 6
|
0 0 6 6 6
|
||||||
0 5 5 1.7917595 5
|
0 5 5 1.79176 5
|
||||||
1 1 2 1.0986123 42
|
1 1 2 1.09861 42
|
||||||
1 1 2 1.0986123 42
|
1 1 2 1.09861 42
|
||||||
2 2 4 1.609438 2
|
2 2 4 1.60944 2
|
||||||
3 3 3 3 3
|
3 3 3 3 3
|
||||||
4 0 4 1.609438 42
|
4 0 4 1.60944 42
|
||||||
{"x":7,"y":7,"a":"14","b":2.7080503,"c":42,"n.a":["1","2"],"n.b":["a","b"]}
|
{"x":7,"y":7,"a":"14","b":2.70805,"c":42,"n.a":["1","2"],"n.b":["a","b"]}
|
||||||
{"x":8,"y":8,"a":"16","b":2.8332133,"c":42,"n.a":["3","4"],"n.c":[0,0],"n.b":["c","d"]}
|
{"x":8,"y":8,"a":"16","b":2.83321,"c":42,"n.a":["3","4"],"n.c":[0,0],"n.b":["c","d"]}
|
||||||
{"x":9,"y":9,"a":"18","b":2.944439,"c":42,"n.a":[],"n.c":[],"n.b":[]}
|
{"x":9,"y":9,"a":"18","b":2.94444,"c":42,"n.a":[],"n.c":[],"n.b":[]}
|
||||||
|
@ -6,7 +6,7 @@ CREATE TABLE defaults
|
|||||||
x UInt32,
|
x UInt32,
|
||||||
y UInt32,
|
y UInt32,
|
||||||
a DEFAULT x + y,
|
a DEFAULT x + y,
|
||||||
b Float32 DEFAULT log(1 + x + y),
|
b Float32 DEFAULT round(log(1 + x + y), 5),
|
||||||
c UInt32 DEFAULT 42,
|
c UInt32 DEFAULT 42,
|
||||||
e MATERIALIZED x + y,
|
e MATERIALIZED x + y,
|
||||||
f ALIAS x + y
|
f ALIAS x + y
|
||||||
|
@ -1,5 +1,3 @@
|
|||||||
USE test;
|
|
||||||
|
|
||||||
DROP TABLE IF EXISTS test1;
|
DROP TABLE IF EXISTS test1;
|
||||||
|
|
||||||
CREATE TABLE test1 (
|
CREATE TABLE test1 (
|
||||||
@ -15,7 +13,7 @@ AS SELECT
|
|||||||
date,
|
date,
|
||||||
zoneId,
|
zoneId,
|
||||||
test
|
test
|
||||||
FROM test.test1;
|
FROM test1;
|
||||||
|
|
||||||
INSERT INTO test1 VALUES ('2018-12-10', '2018-12-10 23:59:59', 1);
|
INSERT INTO test1 VALUES ('2018-12-10', '2018-12-10 23:59:59', 1);
|
||||||
|
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
{"x":0,"y":0,"a":"6","b":6,"c":6}
|
{"x":0,"y":0,"a":"6","b":6,"c":6}
|
||||||
{"x":0,"y":5,"a":"5","b":1.7917595,"c":5}
|
{"x":0,"y":5,"a":"5","b":1.79176,"c":5}
|
||||||
{"x":1,"y":1,"a":"2","b":1.0986123,"c":42}
|
{"x":1,"y":1,"a":"2","b":1.09861,"c":42}
|
||||||
{"x":2,"y":2,"a":"4","b":1.609438,"c":2}
|
{"x":2,"y":2,"a":"4","b":1.60944,"c":2}
|
||||||
{"x":3,"y":3,"a":"3","b":3,"c":3}
|
{"x":3,"y":3,"a":"3","b":3,"c":3}
|
||||||
{"x":4,"y":0,"a":"4","b":1.609438,"c":42}
|
{"x":4,"y":0,"a":"4","b":1.60944,"c":42}
|
||||||
|
@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
. $CURDIR/../shell_config.sh
|
. $CURDIR/../shell_config.sh
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS defaults"
|
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS defaults"
|
||||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE defaults (x UInt32, y UInt32, a DEFAULT x + y, b Float32 DEFAULT log(1 + x + y), c UInt32 DEFAULT 42, e MATERIALIZED x + y, f ALIAS x + y) ENGINE = Memory"
|
$CLICKHOUSE_CLIENT --query="CREATE TABLE defaults (x UInt32, y UInt32, a DEFAULT x + y, b Float32 DEFAULT round(log(1 + x + y), 5), c UInt32 DEFAULT 42, e MATERIALIZED x + y, f ALIAS x + y) ENGINE = Memory"
|
||||||
|
|
||||||
echo -ne '{"x":1, "y":1}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}&query=INSERT%20INTO%20defaults%20FORMAT%20JSONEachRow%20SETTINGS%20input_format_defaults_for_omitted_fields=1" --data-binary @-
|
echo -ne '{"x":1, "y":1}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}&query=INSERT%20INTO%20defaults%20FORMAT%20JSONEachRow%20SETTINGS%20input_format_defaults_for_omitted_fields=1" --data-binary @-
|
||||||
echo -ne '{"x":2, "y":2, "c":2}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}&query=INSERT+INTO+defaults+FORMAT+JSONEachRow+SETTINGS+input_format_defaults_for_omitted_fields=1" --data-binary @-
|
echo -ne '{"x":2, "y":2, "c":2}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}&query=INSERT+INTO+defaults+FORMAT+JSONEachRow+SETTINGS+input_format_defaults_for_omitted_fields=1" --data-binary @-
|
||||||
|
3
dbms/tests/queries/0_stateless/00978_ml_math.reference
Normal file
3
dbms/tests/queries/0_stateless/00978_ml_math.reference
Normal file
@ -0,0 +1,3 @@
|
|||||||
|
0.26894 0.26894 0.26894 -0.76159 -0.76159 -0.76159
|
||||||
|
0.5 0.5 0.5 0 0 0
|
||||||
|
0.73106 0.73106 0.73106 0.76159 0.76159 0.76159
|
4
dbms/tests/queries/0_stateless/00978_ml_math.sql
Normal file
4
dbms/tests/queries/0_stateless/00978_ml_math.sql
Normal file
@ -0,0 +1,4 @@
|
|||||||
|
SELECT
|
||||||
|
round(sigmoid(x), 5), round(sigmoid(toFloat32(x)), 5), round(sigmoid(toFloat64(x)), 5),
|
||||||
|
round(tanh(x), 5), round(TANH(toFloat32(x)), 5), round(TANh(toFloat64(x)), 5)
|
||||||
|
FROM (SELECT arrayJoin([-1, 0, 1]) AS x);
|
17
libs/libglibc-compatibility/musl/__math_divzerof.c
Normal file
17
libs/libglibc-compatibility/musl/__math_divzerof.c
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
#include <stdint.h>
|
||||||
|
|
||||||
|
/* fp_barrier returns its input, but limits code transformations
|
||||||
|
as if it had a side-effect (e.g. observable io) and returned
|
||||||
|
an arbitrary value. */
|
||||||
|
|
||||||
|
static inline float fp_barrierf(float x)
|
||||||
|
{
|
||||||
|
volatile float y = x;
|
||||||
|
return y;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
float __math_divzerof(uint32_t sign)
|
||||||
|
{
|
||||||
|
return fp_barrierf(sign ? -1.0f : 1.0f) / 0.0f;
|
||||||
|
}
|
4
libs/libglibc-compatibility/musl/__math_invalidf.c
Normal file
4
libs/libglibc-compatibility/musl/__math_invalidf.c
Normal file
@ -0,0 +1,4 @@
|
|||||||
|
float __math_invalidf(float x)
|
||||||
|
{
|
||||||
|
return (x - x) / (x - x);
|
||||||
|
}
|
88
libs/libglibc-compatibility/musl/logf.c
Normal file
88
libs/libglibc-compatibility/musl/logf.c
Normal file
@ -0,0 +1,88 @@
|
|||||||
|
/*
|
||||||
|
* Single-precision log function.
|
||||||
|
*
|
||||||
|
* Copyright (c) 2017-2018, Arm Limited.
|
||||||
|
* SPDX-License-Identifier: MIT
|
||||||
|
*/
|
||||||
|
|
||||||
|
#include <math.h>
|
||||||
|
#include <stdint.h>
|
||||||
|
#include "logf_data.h"
|
||||||
|
|
||||||
|
float __math_invalidf(float);
|
||||||
|
float __math_divzerof(uint32_t);
|
||||||
|
|
||||||
|
/*
|
||||||
|
LOGF_TABLE_BITS = 4
|
||||||
|
LOGF_POLY_ORDER = 4
|
||||||
|
|
||||||
|
ULP error: 0.818 (nearest rounding.)
|
||||||
|
Relative error: 1.957 * 2^-26 (before rounding.)
|
||||||
|
*/
|
||||||
|
|
||||||
|
#define T __logf_data.tab
|
||||||
|
#define A __logf_data.poly
|
||||||
|
#define Ln2 __logf_data.ln2
|
||||||
|
#define N (1 << LOGF_TABLE_BITS)
|
||||||
|
#define OFF 0x3f330000
|
||||||
|
#define WANT_ROUNDING 1
|
||||||
|
|
||||||
|
#define asuint(f) ((union{float _f; uint32_t _i;}){f})._i
|
||||||
|
#define asfloat(i) ((union{uint32_t _i; float _f;}){i})._f
|
||||||
|
|
||||||
|
/* Evaluate an expression as the specified type. With standard excess
|
||||||
|
precision handling a type cast or assignment is enough (with
|
||||||
|
-ffloat-store an assignment is required, in old compilers argument
|
||||||
|
passing and return statement may not drop excess precision). */
|
||||||
|
|
||||||
|
static inline float eval_as_float(float x)
|
||||||
|
{
|
||||||
|
float y = x;
|
||||||
|
return y;
|
||||||
|
}
|
||||||
|
|
||||||
|
float logf(float x)
|
||||||
|
{
|
||||||
|
double_t z, r, r2, y, y0, invc, logc;
|
||||||
|
uint32_t ix, iz, tmp;
|
||||||
|
int k, i;
|
||||||
|
|
||||||
|
ix = asuint(x);
|
||||||
|
/* Fix sign of zero with downward rounding when x==1. */
|
||||||
|
if (WANT_ROUNDING && __builtin_expect(ix == 0x3f800000, 0))
|
||||||
|
return 0;
|
||||||
|
if (__builtin_expect(ix - 0x00800000 >= 0x7f800000 - 0x00800000, 0)) {
|
||||||
|
/* x < 0x1p-126 or inf or nan. */
|
||||||
|
if (ix * 2 == 0)
|
||||||
|
return __math_divzerof(1);
|
||||||
|
if (ix == 0x7f800000) /* log(inf) == inf. */
|
||||||
|
return x;
|
||||||
|
if ((ix & 0x80000000) || ix * 2 >= 0xff000000)
|
||||||
|
return __math_invalidf(x);
|
||||||
|
/* x is subnormal, normalize it. */
|
||||||
|
ix = asuint(x * 0x1p23f);
|
||||||
|
ix -= 23 << 23;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* x = 2^k z; where z is in range [OFF,2*OFF] and exact.
|
||||||
|
The range is split into N subintervals.
|
||||||
|
The ith subinterval contains z and c is near its center. */
|
||||||
|
tmp = ix - OFF;
|
||||||
|
i = (tmp >> (23 - LOGF_TABLE_BITS)) % N;
|
||||||
|
k = (int32_t)tmp >> 23; /* arithmetic shift */
|
||||||
|
iz = ix - (tmp & 0x1ff << 23);
|
||||||
|
invc = T[i].invc;
|
||||||
|
logc = T[i].logc;
|
||||||
|
z = (double_t)asfloat(iz);
|
||||||
|
|
||||||
|
/* log(x) = log1p(z/c-1) + log(c) + k*Ln2 */
|
||||||
|
r = z * invc - 1;
|
||||||
|
y0 = logc + (double_t)k * Ln2;
|
||||||
|
|
||||||
|
/* Pipelined polynomial evaluation to approximate log1p(r). */
|
||||||
|
r2 = r * r;
|
||||||
|
y = A[1] * r + A[2];
|
||||||
|
y = A[0] * r2 + y;
|
||||||
|
y = y * r2 + (y0 + r);
|
||||||
|
return eval_as_float(y);
|
||||||
|
}
|
33
libs/libglibc-compatibility/musl/logf_data.c
Normal file
33
libs/libglibc-compatibility/musl/logf_data.c
Normal file
@ -0,0 +1,33 @@
|
|||||||
|
/*
|
||||||
|
* Data definition for logf.
|
||||||
|
*
|
||||||
|
* Copyright (c) 2017-2018, Arm Limited.
|
||||||
|
* SPDX-License-Identifier: MIT
|
||||||
|
*/
|
||||||
|
|
||||||
|
#include "logf_data.h"
|
||||||
|
|
||||||
|
const struct logf_data __logf_data = {
|
||||||
|
.tab = {
|
||||||
|
{ 0x1.661ec79f8f3bep+0, -0x1.57bf7808caadep-2 },
|
||||||
|
{ 0x1.571ed4aaf883dp+0, -0x1.2bef0a7c06ddbp-2 },
|
||||||
|
{ 0x1.49539f0f010bp+0, -0x1.01eae7f513a67p-2 },
|
||||||
|
{ 0x1.3c995b0b80385p+0, -0x1.b31d8a68224e9p-3 },
|
||||||
|
{ 0x1.30d190c8864a5p+0, -0x1.6574f0ac07758p-3 },
|
||||||
|
{ 0x1.25e227b0b8eap+0, -0x1.1aa2bc79c81p-3 },
|
||||||
|
{ 0x1.1bb4a4a1a343fp+0, -0x1.a4e76ce8c0e5ep-4 },
|
||||||
|
{ 0x1.12358f08ae5bap+0, -0x1.1973c5a611cccp-4 },
|
||||||
|
{ 0x1.0953f419900a7p+0, -0x1.252f438e10c1ep-5 },
|
||||||
|
{ 0x1p+0, 0x0p+0 },
|
||||||
|
{ 0x1.e608cfd9a47acp-1, 0x1.aa5aa5df25984p-5 },
|
||||||
|
{ 0x1.ca4b31f026aap-1, 0x1.c5e53aa362eb4p-4 },
|
||||||
|
{ 0x1.b2036576afce6p-1, 0x1.526e57720db08p-3 },
|
||||||
|
{ 0x1.9c2d163a1aa2dp-1, 0x1.bc2860d22477p-3 },
|
||||||
|
{ 0x1.886e6037841edp-1, 0x1.1058bc8a07ee1p-2 },
|
||||||
|
{ 0x1.767dcf5534862p-1, 0x1.4043057b6ee09p-2 },
|
||||||
|
},
|
||||||
|
.ln2 = 0x1.62e42fefa39efp-1,
|
||||||
|
.poly = {
|
||||||
|
-0x1.00ea348b88334p-2, 0x1.5575b0be00b6ap-2, -0x1.ffffef20a4123p-2,
|
||||||
|
}
|
||||||
|
};
|
18
libs/libglibc-compatibility/musl/logf_data.h
Normal file
18
libs/libglibc-compatibility/musl/logf_data.h
Normal file
@ -0,0 +1,18 @@
|
|||||||
|
/*
|
||||||
|
* Copyright (c) 2017-2018, Arm Limited.
|
||||||
|
* SPDX-License-Identifier: MIT
|
||||||
|
*/
|
||||||
|
#ifndef _LOGF_DATA_H
|
||||||
|
#define _LOGF_DATA_H
|
||||||
|
|
||||||
|
#define LOGF_TABLE_BITS 4
|
||||||
|
#define LOGF_POLY_ORDER 4
|
||||||
|
extern __attribute__((__visibility__("hidden"))) const struct logf_data {
|
||||||
|
struct {
|
||||||
|
double invc, logc;
|
||||||
|
} tab[1 << LOGF_TABLE_BITS];
|
||||||
|
double ln2;
|
||||||
|
double poly[LOGF_POLY_ORDER - 1]; /* First order coefficient is 1. */
|
||||||
|
} __logf_data;
|
||||||
|
|
||||||
|
#endif
|
Loading…
Reference in New Issue
Block a user