Merge remote-tracking branch 'upstream/master' into fix25

This commit is contained in:
proller 2019-08-04 10:00:33 +00:00
commit 6c89980158
59 changed files with 630 additions and 313 deletions

5
.gitmodules vendored
View File

@ -93,10 +93,13 @@
url = https://github.com/ClickHouse-Extras/libunwind.git
[submodule "contrib/simdjson"]
path = contrib/simdjson
url = https://github.com/lemire/simdjson.git
url = https://github.com/ClickHouse-Extras/simdjson.git
[submodule "contrib/rapidjson"]
path = contrib/rapidjson
url = https://github.com/Tencent/rapidjson
[submodule "contrib/mimalloc"]
path = contrib/mimalloc
url = https://github.com/ClickHouse-Extras/mimalloc
[submodule "contrib/fastops"]
path = contrib/fastops
url = https://github.com/ClickHouse-Extras/fastops

View File

@ -476,6 +476,7 @@ include (cmake/find_hyperscan.cmake)
include (cmake/find_mimalloc.cmake)
include (cmake/find_simdjson.cmake)
include (cmake/find_rapidjson.cmake)
include (cmake/find_fastops.cmake)
find_contrib_lib(cityhash)
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(readpassphrase)
add_default_dependencies(unwind_static)
add_default_dependencies(fastops)
endif ()

15
cmake/find_fastops.cmake Normal file
View 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")

View File

@ -330,3 +330,7 @@ endif()
if (USE_MIMALLOC)
add_subdirectory (mimalloc)
endif()
if (USE_FASTOPS)
add_subdirectory (fastops-cmake)
endif()

1
contrib/fastops vendored Submodule

@ -0,0 +1 @@
Subproject commit d2c85c5d6549cfd648a7f31ef7b14341881ff8ae

View 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

@ -1 +1 @@
Subproject commit 3bd3116cf8faf6d482dc31423b16533bfa2696f7
Subproject commit e3f6322af762213ff2087ce3366bf9541c7fd355

View File

@ -10,7 +10,7 @@
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/ThreadPool.h>
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
#include <Storages/StorageBlockInserted.h>
#include <Storages/StorageValues.h>
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).
/// InterpreterSelectQuery will do processing of alias columns.
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());
BlockInputStreamPtr in = std::make_shared<MaterializingBlockInputStream>(select.execute().in);

View File

@ -20,6 +20,7 @@ target_link_libraries(clickhouse_functions
${METROHASH_LIBRARIES}
murmurhash
${BASE64_LIBRARY}
${FASTOPS_LIBRARY}
PRIVATE
${ZLIB_LIBRARIES}
@ -31,7 +32,7 @@ if (OPENSSL_CRYPTO_LIBRARY)
endif()
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)
target_include_directories (clickhouse_functions PRIVATE ${CONSISTENT_HASHING_INCLUDE_DIR})
@ -48,7 +49,11 @@ if (USE_ICU)
endif ()
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 ()
if (ENABLE_TESTS)

View File

@ -31,6 +31,14 @@
#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
{
@ -41,16 +49,14 @@ namespace ErrorCodes
template <typename Impl>
class FunctionMathUnaryFloat64 : public IFunction
class FunctionMathUnary : public IFunction
{
public:
static constexpr auto name = Impl::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathUnaryFloat64>(); }
static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration");
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathUnary>(); }
private:
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
@ -60,11 +66,35 @@ private:
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
/// Integers are converted to Float64.
if (Impl::always_returns_float64 || !isFloat(arg))
return std::make_shared<DataTypeFloat64>();
else
return arg;
}
template <typename T>
static void executeInIterations(const T * src_data, Float64 * dst_data, size_t size)
template <typename T, typename ReturnType>
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_size = size - rows_remaining;
@ -77,21 +107,22 @@ private:
T src_remaining[Impl::rows_per_iteration];
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));
Float64 dst_remaining[Impl::rows_per_iteration];
ReturnType dst_remaining[Impl::rows_per_iteration];
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)
{
const auto & src_data = col->getData();
const size_t size = src_data.size();
auto dst = ColumnVector<Float64>::create();
auto dst = ColumnVector<ReturnType>::create();
auto & dst_data = dst->getData();
dst_data.resize(size);
@ -101,19 +132,19 @@ private:
return true;
}
template <typename T>
template <typename T, typename ReturnType>
static bool execute(Block & block, const ColumnDecimal<T> * col, const size_t result)
{
const auto & src_data = col->getData();
const size_t size = src_data.size();
UInt32 scale = src_data.getScale();
auto dst = ColumnVector<Float64>::create();
auto dst = ColumnVector<ReturnType>::create();
auto & dst_data = dst->getData();
dst_data.resize(size);
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);
@ -131,10 +162,11 @@ private:
{
using Types = std::decay_t<decltype(types)>;
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>>;
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))
@ -149,6 +181,7 @@ struct UnaryFunctionPlain
{
static constexpr auto name = Name::name;
static constexpr auto rows_per_iteration = 1;
static constexpr bool always_returns_float64 = true;
template <typename T>
static void execute(const T * src, Float64 * dst)
@ -164,6 +197,7 @@ struct UnaryFunctionVectorized
{
static constexpr auto name = Name::name;
static constexpr auto rows_per_iteration = 2;
static constexpr bool always_returns_float64 = true;
template <typename T>
static void execute(const T * src, Float64 * dst)

View File

@ -25,14 +25,14 @@ struct SimdJSONParser
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",
ErrorCodes::CANNOT_ALLOCATE_MEMORY};
}
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}; }
static bool isInt64(const Iterator & it) { return it.is_integer(); }

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct AcosName { static constexpr auto name = "acos"; };
using FunctionAcos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AcosName, acos>>;
using FunctionAcos = FunctionMathUnary<UnaryFunctionVectorized<AcosName, acos>>;
void registerFunctionAcos(FunctionFactory & factory)
{

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct AsinName { static constexpr auto name = "asin"; };
using FunctionAsin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AsinName, asin>>;
using FunctionAsin = FunctionMathUnary<UnaryFunctionVectorized<AsinName, asin>>;
void registerFunctionAsin(FunctionFactory & factory)
{

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct AtanName { static constexpr auto name = "atan"; };
using FunctionAtan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<AtanName, atan>>;
using FunctionAtan = FunctionMathUnary<UnaryFunctionVectorized<AtanName, atan>>;
void registerFunctionAtan(FunctionFactory & factory)
{

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct CbrtName { static constexpr auto name = "cbrt"; };
using FunctionCbrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CbrtName, cbrt>>;
using FunctionCbrt = FunctionMathUnary<UnaryFunctionVectorized<CbrtName, cbrt>>;
void registerFunctionCbrt(FunctionFactory & factory)
{

View File

@ -9,3 +9,4 @@
#cmakedefine01 USE_SIMDJSON
#cmakedefine01 USE_RAPIDJSON
#cmakedefine01 USE_H3
#cmakedefine01 USE_FASTOPS

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct CosName { static constexpr auto name = "cos"; };
using FunctionCos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CosName, cos>>;
using FunctionCos = FunctionMathUnary<UnaryFunctionVectorized<CosName, cos>>;
void registerFunctionCos(FunctionFactory & factory)
{

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
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)
{

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
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)
{

View File

@ -1,11 +1,34 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
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)
{

View File

@ -1,4 +1,4 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
#if !USE_VECTORCLASS
# include <common/preciseExp10.h>
@ -9,7 +9,7 @@ namespace DB
struct Exp10Name { static constexpr auto name = "exp10"; };
using FunctionExp10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp10Name,
using FunctionExp10 = FunctionMathUnary<UnaryFunctionVectorized<Exp10Name,
#if USE_VECTORCLASS
exp10
#else

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct Exp2Name { static constexpr auto name = "exp2"; };
using FunctionExp2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp2Name, exp2>>;
using FunctionExp2 = FunctionMathUnary<UnaryFunctionVectorized<Exp2Name, exp2>>;
void registerFunctionExp2(FunctionFactory & factory)
{

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
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)
{

View File

@ -1,11 +1,34 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
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)
{

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct Log10Name { static constexpr auto name = "log10"; };
using FunctionLog10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log10Name, log10>>;
using FunctionLog10 = FunctionMathUnary<UnaryFunctionVectorized<Log10Name, log10>>;
void registerFunctionLog10(FunctionFactory & factory)
{

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct Log2Name { static constexpr auto name = "log2"; };
using FunctionLog2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log2Name, log2>>;
using FunctionLog2 = FunctionMathUnary<UnaryFunctionVectorized<Log2Name, log2>>;
void registerFunctionLog2(FunctionFactory & factory)
{

View File

@ -23,6 +23,8 @@ void registerFunctionTan(FunctionFactory & factory);
void registerFunctionAsin(FunctionFactory & factory);
void registerFunctionAcos(FunctionFactory & factory);
void registerFunctionAtan(FunctionFactory & factory);
void registerFunctionSigmoid(FunctionFactory & factory);
void registerFunctionTanh(FunctionFactory & factory);
void registerFunctionPow(FunctionFactory & factory);
void registerFunctionsMath(FunctionFactory & factory)
@ -47,6 +49,8 @@ void registerFunctionsMath(FunctionFactory & factory)
registerFunctionAsin(factory);
registerFunctionAcos(factory);
registerFunctionAtan(factory);
registerFunctionSigmoid(factory);
registerFunctionTanh(factory);
registerFunctionPow(factory);
}

View 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>();
}
}

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct SinName { static constexpr auto name = "sin"; };
using FunctionSin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SinName, sin>>;
using FunctionSin = FunctionMathUnary<UnaryFunctionVectorized<SinName, sin>>;
void registerFunctionSin(FunctionFactory & factory)
{

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct SqrtName { static constexpr auto name = "sqrt"; };
using FunctionSqrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SqrtName, sqrt>>;
using FunctionSqrt = FunctionMathUnary<UnaryFunctionVectorized<SqrtName, sqrt>>;
void registerFunctionSqrt(FunctionFactory & factory)
{

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct TanName { static constexpr auto name = "tan"; };
using FunctionTan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<TanName, tan>>;
using FunctionTan = FunctionMathUnary<UnaryFunctionVectorized<TanName, tan>>;
void registerFunctionTan(FunctionFactory & factory)
{

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

View File

@ -1,11 +1,11 @@
#include <Functions/FunctionMathUnaryFloat64.h>
#include <Functions/FunctionMathUnary.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
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)
{

View File

@ -133,7 +133,7 @@ private:
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.
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.
Context * query_context = nullptr;
Context * session_context = nullptr; /// Session context or nullptr. Could be equal to this.

View File

@ -48,9 +48,7 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
#include <Storages/IStorage.h>
#include <Storages/StorageBlockInserted.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/StorageValues.h>
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFactory.h>
@ -274,8 +272,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (auto view_source = context.getViewSource())
{
auto & storage_block = static_cast<const StorageBlockInserted &>(*view_source);
if (storage_block.getDatabaseName() == database_name && storage_block.getTableName() == table_name)
auto & storage_values = static_cast<const StorageValues &>(*view_source);
if (storage_values.getDatabaseName() == database_name && storage_values.getTableName() == table_name)
{
/// Read from view source.
storage = context.getViewSource();

View File

@ -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_)
{
}
};
}

View File

@ -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_)
{
}
};
}

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

View File

@ -32,16 +32,16 @@ select tgamma(2) = 1;
select tgamma(3) = 2;
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 log(0) = -inf;
select log(1) = 0;
select log(e()) = 1;
select log(exp(1)) = 1;
select log(exp(2)) = 2;
select sum(abs(log(exp(x)) - x) < 1.0e-9) / count() from system.one array join range(100) as x;
select abs(log(e()) - 1) < 1e-8;
select abs(log(exp(1)) - 1) < 1e-8;
select abs(log(exp(2)) - 2) < 1e-8;
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(0) = 1;

View File

@ -1,70 +1,70 @@
1 1.00 B 1 1.00 B 1 1.00 B
2.718282 2.72 B 2 2.00 B 2 2.00 B
7.389056 7.39 B 7 7.00 B 7 7.00 B
20.085537 20.09 B 20 20.00 B 20 20.00 B
54.59815 54.60 B 54 54.00 B 54 54.00 B
148.413159 148.41 B 148 148.00 B 148 148.00 B
403.428793 403.43 B 403 403.00 B 403 403.00 B
1096.633158 1.07 KiB 1096 1.07 KiB 1096 1.07 KiB
2980.957987 2.91 KiB 2980 2.91 KiB 2980 2.91 KiB
8103.083928 7.91 KiB 8103 7.91 KiB 8103 7.91 KiB
22026.465795 21.51 KiB 22026 21.51 KiB 22026 21.51 KiB
59874.141715 58.47 KiB 59874 58.47 KiB 59874 58.47 KiB
162754.791419 158.94 KiB 162754 158.94 KiB 162754 158.94 KiB
442413.392009 432.04 KiB 442413 432.04 KiB 442413 432.04 KiB
1202604.284165 1.15 MiB 1202604 1.15 MiB 1202604 1.15 MiB
3269017.372472 3.12 MiB 3269017 3.12 MiB 3269017 3.12 MiB
8886110.520508 8.47 MiB 8886110 8.47 MiB 8886110 8.47 MiB
24154952.753575 23.04 MiB 24154952 23.04 MiB 24154952 23.04 MiB
65659969.137331 62.62 MiB 65659969 62.62 MiB 65659969 62.62 MiB
178482300.963187 170.21 MiB 178482300 170.21 MiB 178482300 170.21 MiB
485165195.40979 462.69 MiB 485165195 462.69 MiB 485165195 462.69 MiB
1318815734.483214 1.23 GiB 1318815734 1.23 GiB 1318815734 1.23 GiB
3584912846.131592 3.34 GiB 3584912846 3.34 GiB -710054450 -677.16 MiB
9744803446.248903 9.08 GiB 9744803446 9.08 GiB 1154868854 1.08 GiB
26489122129.84347 24.67 GiB 26489122129 24.67 GiB 719318353 686.00 MiB
72004899337.38588 67.06 GiB 72004899337 67.06 GiB -1009544695 -962.78 MiB
195729609428.83878 182.29 GiB 195729609428 182.29 GiB -1838886188 -1.71 GiB
532048240601.79865 495.51 GiB 532048240601 495.51 GiB -527704103 -503.26 MiB
1446257064291.475 1.32 TiB 1446257064291 1.32 TiB -1146914461 -1.07 GiB
3931334297144.042 3.58 TiB 3931334297144 3.58 TiB 1439221304 1.34 GiB
10686474581524.463 9.72 TiB 10686474581524 9.72 TiB 595949076 568.34 MiB
29048849665247.426 26.42 TiB 29048849665247 26.42 TiB 1985842399 1.85 GiB
78962960182680.69 71.82 TiB 78962960182680 71.82 TiB -13554280 -12.93 MiB
214643579785916.06 195.22 TiB 214643579785916 195.22 TiB -1705798980 -1.59 GiB
583461742527454.9 530.66 TiB 583461742527454 530.66 TiB -974699554 -929.55 MiB
1586013452313430.8 1.41 PiB 1586013452313430 1.41 PiB -2005982378 -1.87 GiB
4311231547115195 3.83 PiB 4311231547115195 3.83 PiB -790034757 -753.44 MiB
11719142372802612 10.41 PiB 11719142372802612 10.41 PiB 1983119412 1.85 GiB
31855931757113756 28.29 PiB 31855931757113756 28.29 PiB 408891804 389.95 MiB
86593400423993740 76.91 PiB 86593400423993744 76.91 PiB 673862032 642.64 MiB
235385266837020000 209.06 PiB 235385266837020000 209.06 PiB 791567712 754.90 MiB
639843493530054900 568.30 PiB 639843493530054912 568.30 PiB 1874080000 1.75 GiB
1739274941520501000 1.51 EiB 1739274941520500992 1.51 EiB 538007808 513.08 MiB
4727839468229346000 4.10 EiB 4727839468229346304 4.10 EiB 2061616128 1.92 GiB
12851600114359308000 11.15 EiB 12851600114359308288 11.15 EiB -1681813504 -1.57 GiB
34934271057485095000 30.30 EiB 0 0.00 B 0 0.00 B
94961194206024480000 82.37 EiB 0 0.00 B 0 0.00 B
258131288619006750000 223.89 EiB 0 0.00 B 0 0.00 B
701673591209763100000 608.60 EiB 0 0.00 B 0 0.00 B
1.9073465724950998e21 1.62 ZiB 0 0.00 B 0 0.00 B
5.184705528587072e21 4.39 ZiB 0 0.00 B 0 0.00 B
1.4093490824269389e22 11.94 ZiB 0 0.00 B 0 0.00 B
3.831008000716577e22 32.45 ZiB 0 0.00 B 0 0.00 B
1.0413759433029089e23 88.21 ZiB 0 0.00 B 0 0.00 B
2.830753303274694e23 239.77 ZiB 0 0.00 B 0 0.00 B
7.694785265142018e23 651.77 ZiB 0 0.00 B 0 0.00 B
2.091659496012996e24 1.73 YiB 0 0.00 B 0 0.00 B
5.685719999335932e24 4.70 YiB 0 0.00 B 0 0.00 B
1.545538935590104e25 12.78 YiB 0 0.00 B 0 0.00 B
4.2012104037905144e25 34.75 YiB 0 0.00 B 0 0.00 B
1.1420073898156842e26 94.46 YiB 0 0.00 B 0 0.00 B
3.10429793570192e26 256.78 YiB 0 0.00 B 0 0.00 B
8.438356668741454e26 698.00 YiB 0 0.00 B 0 0.00 B
2.29378315946961e27 1897.37 YiB 0 0.00 B 0 0.00 B
6.235149080811617e27 5157.59 YiB 0 0.00 B 0 0.00 B
1.6948892444103338e28 14019.80 YiB 0 0.00 B 0 0.00 B
4.607186634331292e28 38109.75 YiB 0 0.00 B 0 0.00 B
1.2523631708422137e29 103593.05 YiB 0 0.00 B 0 0.00 B
3.404276049931741e29 281595.11 YiB 0 0.00 B 0 0.00 B
9.253781725587787e29 765454.88 YiB 0 0.00 B 0 0.00 B
1.00 B 1.00 B 1.00 B
2.72 B 2.00 B 2.00 B
7.39 B 7.00 B 7.00 B
20.09 B 20.00 B 20.00 B
54.60 B 54.00 B 54.00 B
148.41 B 148.00 B 148.00 B
403.43 B 403.00 B 403.00 B
1.07 KiB 1.07 KiB 1.07 KiB
2.91 KiB 2.91 KiB 2.91 KiB
7.91 KiB 7.91 KiB 7.91 KiB
21.51 KiB 21.51 KiB 21.51 KiB
58.47 KiB 58.47 KiB 58.47 KiB
158.94 KiB 158.94 KiB 158.94 KiB
432.04 KiB 432.04 KiB 432.04 KiB
1.15 MiB 1.15 MiB 1.15 MiB
3.12 MiB 3.12 MiB 3.12 MiB
8.47 MiB 8.47 MiB 8.47 MiB
23.04 MiB 23.04 MiB 23.04 MiB
62.62 MiB 62.62 MiB 62.62 MiB
170.21 MiB 170.21 MiB 170.21 MiB
462.69 MiB 462.69 MiB 462.69 MiB
1.23 GiB 1.23 GiB 1.23 GiB
3.34 GiB 3.34 GiB -2.00 GiB
9.08 GiB 9.08 GiB -2.00 GiB
24.67 GiB 24.67 GiB -2.00 GiB
67.06 GiB 67.06 GiB -2.00 GiB
182.29 GiB 182.29 GiB -2.00 GiB
495.51 GiB 495.51 GiB -2.00 GiB
1.32 TiB 1.32 TiB -2.00 GiB
3.58 TiB 3.58 TiB -2.00 GiB
9.72 TiB 9.72 TiB -2.00 GiB
26.42 TiB 26.42 TiB -2.00 GiB
71.82 TiB 71.82 TiB -2.00 GiB
195.22 TiB 195.22 TiB -2.00 GiB
530.66 TiB 530.66 TiB -2.00 GiB
1.41 PiB 1.41 PiB -2.00 GiB
3.83 PiB 3.83 PiB -2.00 GiB
10.41 PiB 10.41 PiB -2.00 GiB
28.29 PiB 28.29 PiB -2.00 GiB
76.91 PiB 76.91 PiB -2.00 GiB
209.06 PiB 209.06 PiB -2.00 GiB
568.30 PiB 568.30 PiB -2.00 GiB
1.51 EiB 1.51 EiB -2.00 GiB
4.10 EiB 4.10 EiB -2.00 GiB
11.15 EiB 11.15 EiB -2.00 GiB
30.30 EiB 0.00 B -2.00 GiB
82.37 EiB 0.00 B -2.00 GiB
223.89 EiB 0.00 B -2.00 GiB
608.60 EiB 0.00 B -2.00 GiB
1.62 ZiB 0.00 B -2.00 GiB
4.39 ZiB 0.00 B -2.00 GiB
11.94 ZiB 0.00 B -2.00 GiB
32.45 ZiB 0.00 B -2.00 GiB
88.21 ZiB 0.00 B -2.00 GiB
239.77 ZiB 0.00 B -2.00 GiB
651.77 ZiB 0.00 B -2.00 GiB
1.73 YiB 0.00 B -2.00 GiB
4.70 YiB 0.00 B -2.00 GiB
12.78 YiB 0.00 B -2.00 GiB
34.75 YiB 0.00 B -2.00 GiB
94.46 YiB 0.00 B -2.00 GiB
256.78 YiB 0.00 B -2.00 GiB
698.00 YiB 0.00 B -2.00 GiB
1897.37 YiB 0.00 B -2.00 GiB
5157.59 YiB 0.00 B -2.00 GiB
14019.80 YiB 0.00 B -2.00 GiB
38109.75 YiB 0.00 B -2.00 GiB
103593.05 YiB 0.00 B -2.00 GiB
281595.11 YiB 0.00 B -2.00 GiB
765454.88 YiB 0.00 B -2.00 GiB

View File

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

View File

@ -1,2 +1,2 @@
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;

View File

@ -1,6 +1,6 @@
42.4200 3.7476 42.419153766068966
42.4200 5.4066 42.41786197045111
42.4200 1.6275 42.413098391048806
42.4200 3.7476 42.419154
42.4200 5.4066 42.417862
42.4200 1.6275 42.413098
42.4200 6.513 42.419169
42.4200 3.4875 42.417263671875
1.00000 0.8427007929497149 0.15729920705028513
@ -8,9 +8,9 @@
0.00 0 1 0
3.14159265 0 -1 -0
1.00 1.5707963267948966 0 0.7853981633974483
42.4200 3.7476 42.419153766068966
42.4200 5.4066 42.41786197045111
42.4200 1.6275 42.413098391048806
42.4200 3.7476 42.419154
42.4200 5.4066 42.417862
42.4200 1.6275 42.413098
42.4200 6.513 42.419169
42.4200 3.4875 42.417263671875
1.00000 0.8427007929497149 0.15729920705028513
@ -18,9 +18,9 @@
0.00 0 1 0
3.14159265358979328 0 -1 -0
1.00 1.5707963267948966 0 0.7853981633974483
42.4200 3.7476 42.419153766068966
42.4200 5.4066 42.41786197045111
42.4200 1.6275 42.413098391048806
42.4200 3.7476 42.419154
42.4200 5.4066 42.417862
42.4200 1.6275 42.413098
42.4200 6.513 42.419169
42.4200 3.4875 42.417263671875
1.00000 0.8427007929497149 0.15729920705028513

View File

@ -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(log2(x), 4) AS y, exp2(y);
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, exp10(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, round(exp2(y), 6);
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(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 toDecimal64('42.42', 4) AS x, toDecimal32(log(x), 4) AS y, exp(y);
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(log2(x), 4) AS y, exp2(y);
SELECT toDecimal64('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, exp10(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, round(exp2(y), 6);
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(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 toDecimal128('42.42', 4) AS x, toDecimal32(log(x), 4) AS y, exp(y);
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(log2(x), 4) AS y, exp2(y);
SELECT toDecimal128('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, exp10(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, round(exp2(y), 6);
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(cbrt(x), 4) AS y, toDecimal64(y, 4) * y * y;

View File

@ -19,7 +19,7 @@
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 0 nan nan nan nan nan nan nan
nan nan nan nan nan
nan nan nan nan nan nan nan
-1 1

View File

@ -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, 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, sin(x), cos(x), tan(x), asin(x), acos(x), atan(x);

View File

@ -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(1);
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);

View File

@ -1,10 +1,10 @@
0 0 6 6 6
0 5 5 1.7917595 5
1 1 2 1.0986123 42
1 1 2 1.0986123 42
2 2 4 1.609438 2
0 5 5 1.79176 5
1 1 2 1.09861 42
1 1 2 1.09861 42
2 2 4 1.60944 2
3 3 3 3 3
4 0 4 1.609438 42
{"x":7,"y":7,"a":"14","b":2.7080503,"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":9,"y":9,"a":"18","b":2.944439,"c":42,"n.a":[],"n.c":[],"n.b":[]}
4 0 4 1.60944 42
{"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.83321,"c":42,"n.a":["3","4"],"n.c":[0,0],"n.b":["c","d"]}
{"x":9,"y":9,"a":"18","b":2.94444,"c":42,"n.a":[],"n.c":[],"n.b":[]}

View File

@ -6,7 +6,7 @@ CREATE TABLE defaults
x UInt32,
y UInt32,
a DEFAULT x + y,
b Float32 DEFAULT log(1 + x + y),
b Float32 DEFAULT round(log(1 + x + y), 5),
c UInt32 DEFAULT 42,
e MATERIALIZED x + y,
f ALIAS x + y

View File

@ -1,5 +1,3 @@
USE test;
DROP TABLE IF EXISTS test1;
CREATE TABLE test1 (
@ -15,7 +13,7 @@ AS SELECT
date,
zoneId,
test
FROM test.test1;
FROM test1;
INSERT INTO test1 VALUES ('2018-12-10', '2018-12-10 23:59:59', 1);

View File

@ -1,6 +1,6 @@
{"x":0,"y":0,"a":"6","b":6,"c":6}
{"x":0,"y":5,"a":"5","b":1.7917595,"c":5}
{"x":1,"y":1,"a":"2","b":1.0986123,"c":42}
{"x":2,"y":2,"a":"4","b":1.609438,"c":2}
{"x":0,"y":5,"a":"5","b":1.79176,"c":5}
{"x":1,"y":1,"a":"2","b":1.09861,"c":42}
{"x":2,"y":2,"a":"4","b":1.60944,"c":2}
{"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}

View File

@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$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":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 @-

View 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

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

View 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;
}

View File

@ -0,0 +1,4 @@
float __math_invalidf(float x)
{
return (x - x) / (x - x);
}

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

View 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,
}
};

View 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