Merge with master

This commit is contained in:
alesapin 2019-10-24 12:33:40 +03:00
commit a6199b7e69
89 changed files with 3012 additions and 616 deletions

3
.github/CODEOWNERS vendored Normal file
View File

@ -0,0 +1,3 @@
dbms/* @ClickHouse/core-assigner
docs/* @ClickHouse/docs
docs/zh/* @ClickHouse/docs-zh

View File

@ -1 +0,0 @@
pr-feature: "New Feature"

23
.github/labeler.yml vendored
View File

@ -1,23 +0,0 @@
# Build changes
pr-build:
- "**/CMakeLists.txt"
# Documentation PRs
documentation:
- "**/*.md"
- "docs/**/*"
pr-documentation:
- "**/*.md"
- "docs/**/*"
# Component labels
comp-mutations:
- "**/*Mutation*"
comp-matview:
- "**/*MaterializedView*"
comp-skipidx:
- "**/*Indices*"
comp-kafka:
- "dbms/src/Storages/Kafka/**/*"
- "dbms/tests/integration/test_storage_kafka/**/*"
- "utils/kafka/**/*"

View File

@ -1,11 +0,0 @@
name: "Pull Request Labeler"
on:
pull_request
jobs:
by-filename:
runs-on: ubuntu-latest
steps:
- uses: "actions/labeler@v2"
with:
repo-token: "${{ secrets.GITHUB_TOKEN }}"

View File

@ -1,13 +1,20 @@
macro(add_glob cur_list)
if (CMAKE_VERSION VERSION_GREATER_EQUAL "3.12")
macro(add_glob cur_list)
file(GLOB __tmp RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} CONFIGURE_DEPENDS ${ARGN})
list(APPEND ${cur_list} ${__tmp})
endmacro()
else ()
macro(add_glob cur_list)
file(GLOB __tmp RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${ARGN})
list(APPEND ${cur_list} ${__tmp})
endmacro()
endmacro()
endif ()
macro(add_headers_and_sources prefix common_path)
add_glob(${prefix}_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
add_glob(${prefix}_headers ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
add_glob(${prefix}_sources ${common_path}/*.cpp ${common_path}/*.c ${common_path}/*.h)
endmacro()
macro(add_headers_only prefix common_path)
add_glob(${prefix}_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
add_glob(${prefix}_headers ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
endmacro()

View File

@ -153,12 +153,10 @@ add_subdirectory(src/Common/Config)
set (all_modules)
macro(add_object_library name common_path)
if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES)
add_glob(dbms_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
add_glob(dbms_sources ${common_path}/*.cpp ${common_path}/*.c ${common_path}/*.h)
add_headers_and_sources(dbms ${common_path})
else ()
list (APPEND all_modules ${name})
add_glob(${name}_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
add_glob(${name}_sources ${common_path}/*.cpp ${common_path}/*.c ${common_path}/*.h)
add_headers_and_sources(${name} ${common_path})
add_library(${name} SHARED ${${name}_sources} ${${name}_headers})
target_link_libraries (${name} PRIVATE -Wl,--unresolved-symbols=ignore-all)
endif ()

View File

@ -1,11 +1,11 @@
# This strings autochanged from release_lib.sh:
set(VERSION_REVISION 54427)
set(VERSION_REVISION 54428)
set(VERSION_MAJOR 19)
set(VERSION_MINOR 16)
set(VERSION_MINOR 17)
set(VERSION_PATCH 1)
set(VERSION_GITHASH 38f65a6a2120d2e76bcf71131068f41195149dfc)
set(VERSION_DESCRIBE v19.16.1.1-prestable)
set(VERSION_STRING 19.16.1.1)
set(VERSION_GITHASH 5286d0afb285a5fbf3d320af3daa6de6b1841374)
set(VERSION_DESCRIBE v19.17.1.1-prestable)
set(VERSION_STRING 19.17.1.1)
# end of autochange
set(VERSION_EXTRA "" CACHE STRING "")

View File

@ -365,7 +365,7 @@ private:
Stopwatch watch;
RemoteBlockInputStream stream(
*(*connection_entries[connection_index]),
query, {}, global_context, &settings, nullptr, Tables(), query_processing_stage);
query, {}, global_context, &settings, nullptr, Scalars(), Tables(), query_processing_stage);
Progress progress;
stream.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); });

View File

@ -19,6 +19,7 @@
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/Net/HTMLForm.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNullable.h>
#include <IO/WriteBufferFromHTTPServerResponse.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ParserQueryWithOutput.h>
@ -154,9 +155,19 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
SQLSMALLINT type = 0;
/// TODO Why 301?
SQLCHAR column_name[301];
/// TODO Result is not checked.
POCO_SQL_ODBC_CLASS::SQLDescribeCol(hstmt, ncol, column_name, sizeof(column_name), nullptr, &type, nullptr, nullptr, nullptr);
columns.emplace_back(reinterpret_cast<char *>(column_name), getDataType(type));
SQLSMALLINT nullable;
const auto result = POCO_SQL_ODBC_CLASS::SQLDescribeCol(hstmt, ncol, column_name, sizeof(column_name), nullptr, &type, nullptr, nullptr, &nullable);
if (POCO_SQL_ODBC_CLASS::Utility::isError(result))
throw POCO_SQL_ODBC_CLASS::StatementException(hstmt);
auto column_type = getDataType(type);
if (nullable == SQL_NULLABLE)
{
column_type = std::make_shared<DataTypeNullable>(column_type);
}
columns.emplace_back(reinterpret_cast<char *>(column_name), std::move(column_type));
}
WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout);

View File

@ -850,9 +850,10 @@ bool TCPHandler::receivePacket()
return true;
case Protocol::Client::Data:
case Protocol::Client::Scalar:
if (state.empty())
receiveUnexpectedData();
return receiveData();
return receiveData(packet_type == Protocol::Client::Scalar);
case Protocol::Client::Ping:
writeVarUInt(Protocol::Server::Pong, *out);
@ -957,18 +958,22 @@ void TCPHandler::receiveUnexpectedQuery()
throw NetException("Unexpected packet Query received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
}
bool TCPHandler::receiveData()
bool TCPHandler::receiveData(bool scalar)
{
initBlockInput();
/// The name of the temporary table for writing data, default to empty string
String external_table_name;
readStringBinary(external_table_name, *in);
String name;
readStringBinary(name, *in);
/// Read one block from the network and write it down
Block block = state.block_in->read();
if (block)
{
if (scalar)
query_context->addScalar(name, block);
else
{
/// If there is an insert request, then the data should be written directly to `state.io.out`.
/// Otherwise, we write the blocks in the temporary `external_table_name` table.
@ -976,12 +981,12 @@ bool TCPHandler::receiveData()
{
StoragePtr storage;
/// If such a table does not exist, create it.
if (!(storage = query_context->tryGetExternalTable(external_table_name)))
if (!(storage = query_context->tryGetExternalTable(name)))
{
NamesAndTypesList columns = block.getNamesAndTypesList();
storage = StorageMemory::create("_external", external_table_name, ColumnsDescription{columns}, ConstraintsDescription{});
storage = StorageMemory::create("_external", name, ColumnsDescription{columns}, ConstraintsDescription{});
storage->startup();
query_context->addExternalTable(external_table_name, storage);
query_context->addExternalTable(name, storage);
}
/// The data will be written directly to the table.
state.io.out = storage->write(ASTPtr(), *query_context);
@ -990,6 +995,7 @@ bool TCPHandler::receiveData()
state.block_for_input = block;
else
state.io.out->write(block);
}
return true;
}
else

View File

@ -153,7 +153,7 @@ private:
void receiveHello();
bool receivePacket();
void receiveQuery();
bool receiveData();
bool receiveData(bool scalar);
bool readDataNext(const size_t & poll_interval, const int & receive_timeout);
void readData(const Settings & global_settings);
std::tuple<size_t, int> getReadTimeouts(const Settings & global_settings);

View File

@ -31,9 +31,9 @@ struct AggregateFunctionAvgData
if constexpr (std::numeric_limits<ResultT>::is_iec559)
return static_cast<ResultT>(sum) / count; /// allow division by zero
if (!count)
throw Exception("AggregateFunctionAvg with zero values", ErrorCodes::LOGICAL_ERROR);
return static_cast<ResultT>(sum) / count;
if (count == 0)
return static_cast<ResultT>(0);
return static_cast<ResultT>(sum / count);
}
};
@ -43,10 +43,10 @@ template <typename T, typename Data>
class AggregateFunctionAvg final : public IAggregateFunctionDataHelper<Data, AggregateFunctionAvg<T, Data>>
{
public:
using ResultType = std::conditional_t<IsDecimalNumber<T>, Decimal128, Float64>;
using ResultDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<Decimal128>, DataTypeNumber<Float64>>;
using ResultType = std::conditional_t<IsDecimalNumber<T>, T, Float64>;
using ResultDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<T>, DataTypeNumber<Float64>>;
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<Decimal128>, ColumnVector<Float64>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<Float64>>;
/// ctor for native types
AggregateFunctionAvg(const DataTypes & argument_types_)

View File

@ -109,8 +109,11 @@ public:
double quantileInterpolated(double level)
{
if (samples.empty())
{
if (DB::IsDecimalNumber<T>)
return 0;
return onEmpty<double>();
}
sortIfNeeded();
double index = std::max(0., std::min(samples.size() - 1., level * (samples.size() - 1)));

View File

@ -30,6 +30,7 @@
namespace CurrentMetrics
{
extern const Metric SendScalars;
extern const Metric SendExternalTables;
}
@ -441,7 +442,7 @@ void Connection::sendCancel()
}
void Connection::sendData(const Block & block, const String & name)
void Connection::sendData(const Block & block, const String & name, bool scalar)
{
//LOG_TRACE(log_wrapper.get(), "Sending data");
@ -455,6 +456,9 @@ void Connection::sendData(const Block & block, const String & name)
block_out = std::make_shared<NativeBlockOutputStream>(*maybe_compressed_out, server_revision, block.cloneEmpty());
}
if (scalar)
writeVarUInt(Protocol::Client::Scalar, *out);
else
writeVarUInt(Protocol::Client::Data, *out);
writeStringBinary(name, *out);
@ -484,6 +488,44 @@ void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String
}
void Connection::sendScalarsData(Scalars & data)
{
if (data.empty())
return;
Stopwatch watch;
size_t out_bytes = out ? out->count() : 0;
size_t maybe_compressed_out_bytes = maybe_compressed_out ? maybe_compressed_out->count() : 0;
size_t rows = 0;
CurrentMetrics::Increment metric_increment{CurrentMetrics::SendScalars};
for (auto & elem : data)
{
rows += elem.second.rows();
sendData(elem.second, elem.first, true /* scalar */);
}
out_bytes = out->count() - out_bytes;
maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes;
double elapsed = watch.elapsedSeconds();
std::stringstream msg;
msg << std::fixed << std::setprecision(3);
msg << "Sent data for " << data.size() << " scalars, total " << rows << " rows in " << elapsed << " sec., "
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., "
<< maybe_compressed_out_bytes / 1048576.0 << " MiB (" << maybe_compressed_out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
if (compression == Protocol::Compression::Enable)
msg << ", compressed " << static_cast<double>(maybe_compressed_out_bytes) / out_bytes << " times to "
<< out_bytes / 1048576.0 << " MiB (" << out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
else
msg << ", no compression.";
LOG_DEBUG(log_wrapper.get(), msg.rdbuf());
}
void Connection::sendExternalTablesData(ExternalTablesData & data)
{
if (data.empty())

View File

@ -133,7 +133,9 @@ public:
void sendCancel();
/// Send block of data; if name is specified, server will write it to external (temporary) table of that name.
void sendData(const Block & block, const String & name = "");
void sendData(const Block & block, const String & name = "", bool scalar = false);
/// Send all scalars.
void sendScalarsData(Scalars & data);
/// Send all contents of external (temporary) tables.
void sendExternalTablesData(ExternalTablesData & data);

View File

@ -51,6 +51,21 @@ MultiplexedConnections::MultiplexedConnections(
active_connection_count = connections.size();
}
void MultiplexedConnections::sendScalarsData(Scalars & data)
{
std::lock_guard lock(cancel_mutex);
if (!sent_query)
throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR);
for (ReplicaState & state : replica_states)
{
Connection * connection = state.connection;
if (connection != nullptr)
connection->sendScalarsData(data);
}
}
void MultiplexedConnections::sendExternalTablesData(std::vector<ExternalTablesData> & data)
{
std::lock_guard lock(cancel_mutex);

View File

@ -27,6 +27,8 @@ public:
std::vector<IConnectionPool::Entry> && connections,
const Settings & settings_, const ThrottlerPtr & throttler_);
/// Send all scalars to replicas.
void sendScalarsData(Scalars & data);
/// Send all content of external tables to replicas.
void sendExternalTablesData(std::vector<ExternalTablesData> & data);

View File

@ -21,6 +21,7 @@
M(OpenFileForWrite, "Number of files open for writing") \
M(Read, "Number of read (read, pread, io_getevents, etc.) syscalls in fly") \
M(Write, "Number of write (write, pwrite, io_getevents, etc.) syscalls in fly") \
M(SendScalars, "Number of connections that are sending data for scalars to remote servers.") \
M(SendExternalTables, "Number of connections that are sending data for external tables to remote servers. External tables are used to implement GLOBAL IN and GLOBAL JOIN operators with distributed subqueries.") \
M(QueryThread, "Number of query processing threads") \
M(ReadonlyReplica, "Number of Replicated tables that are currently in readonly state due to re-initialization after ZooKeeper session loss or due to startup without ZooKeeper configured.") \

View File

@ -292,7 +292,7 @@ Volume::Volume(
formatReadableSizeWithBinarySuffix(max_data_part_size) << ")");
}
constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u;
if (max_data_part_size < MIN_PART_SIZE)
if (max_data_part_size != 0 && max_data_part_size < MIN_PART_SIZE)
LOG_WARNING(logger, "Volume " << backQuote(name) << " max_data_part_size is too low ("
<< formatReadableSizeWithBinarySuffix(max_data_part_size) << " < "
<< formatReadableSizeWithBinarySuffix(MIN_PART_SIZE) << ")");

View File

@ -459,9 +459,11 @@ namespace ErrorCodes
extern const int DICTIONARY_ACCESS_DENIED = 482;
extern const int TOO_MANY_REDIRECTS = 483;
extern const int INTERNAL_REDIS_ERROR = 484;
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY = 485;
extern const int UNKNOWN_DICTIONARY = 486;
extern const int INCORRECT_DICTIONARY_DEFINITION = 487;
extern const int SCALAR_ALREADY_EXISTS = 485;
extern const int UNKNOWN_SCALAR = 486;
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY = 487;
extern const int UNKNOWN_DICTIONARY = 488;
extern const int INCORRECT_DICTIONARY_DEFINITION = 489;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -358,6 +358,12 @@ protected:
template <typename, typename, typename, typename, typename, typename, size_t>
friend class TwoLevelHashTable;
template <typename, typename, size_t>
friend class TwoLevelStringHashTable;
template <typename SubMaps>
friend class StringHashTable;
using HashValue = size_t;
using Self = HashTable;
using cell_type = Cell;

View File

@ -0,0 +1,180 @@
#pragma once
#include <Common/HashTable/HashMap.h>
#include <Common/HashTable/HashTableAllocator.h>
#include <Common/HashTable/StringHashTable.h>
template <typename Key, typename TMapped>
struct StringHashMapCell : public HashMapCell<Key, TMapped, StringHashTableHash, HashTableNoState>
{
using Base = HashMapCell<Key, TMapped, StringHashTableHash, HashTableNoState>;
using Base::Base;
static constexpr bool need_zero_value_storage = false;
};
template<typename Key, typename Mapped>
auto lookupResultGetMapped(StringHashMapCell<Key, Mapped> * cell) { return &cell->getSecond(); }
template <typename TMapped>
struct StringHashMapCell<StringKey16, TMapped> : public HashMapCell<StringKey16, TMapped, StringHashTableHash, HashTableNoState>
{
using Base = HashMapCell<StringKey16, TMapped, StringHashTableHash, HashTableNoState>;
using Base::Base;
static constexpr bool need_zero_value_storage = false;
bool isZero(const HashTableNoState & state) const { return isZero(this->value.first, state); }
// Assuming String does not contain zero bytes. NOTE: Cannot be used in serialized method
static bool isZero(const StringKey16 & key, const HashTableNoState & /*state*/) { return key.low == 0; }
void setZero() { this->value.first.low = 0; }
};
template <typename TMapped>
struct StringHashMapCell<StringKey24, TMapped> : public HashMapCell<StringKey24, TMapped, StringHashTableHash, HashTableNoState>
{
using Base = HashMapCell<StringKey24, TMapped, StringHashTableHash, HashTableNoState>;
using Base::Base;
static constexpr bool need_zero_value_storage = false;
bool isZero(const HashTableNoState & state) const { return isZero(this->value.first, state); }
// Assuming String does not contain zero bytes. NOTE: Cannot be used in serialized method
static bool isZero(const StringKey24 & key, const HashTableNoState & /*state*/) { return key.a == 0; }
void setZero() { this->value.first.a = 0; }
};
template <typename TMapped>
struct StringHashMapCell<StringRef, TMapped> : public HashMapCellWithSavedHash<StringRef, TMapped, StringHashTableHash, HashTableNoState>
{
using Base = HashMapCellWithSavedHash<StringRef, TMapped, StringHashTableHash, HashTableNoState>;
using Base::Base;
static constexpr bool need_zero_value_storage = false;
};
template <typename TMapped, typename Allocator>
struct StringHashMapSubMaps
{
using T0 = StringHashTableEmpty<StringHashMapCell<StringRef, TMapped>>;
using T1 = HashMapTable<StringKey8, StringHashMapCell<StringKey8, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
using T2 = HashMapTable<StringKey16, StringHashMapCell<StringKey16, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
using T3 = HashMapTable<StringKey24, StringHashMapCell<StringKey24, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
using Ts = HashMapTable<StringRef, StringHashMapCell<StringRef, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
};
template <typename TMapped, typename Allocator = HashTableAllocator>
class StringHashMap : public StringHashTable<StringHashMapSubMaps<TMapped, Allocator>>
{
public:
using Base = StringHashTable<StringHashMapSubMaps<TMapped, Allocator>>;
using Self = StringHashMap;
using Key = StringRef;
using key_type = StringRef;
using mapped_type = TMapped;
using value_type = typename Base::Ts::value_type;
using LookupResult = mapped_type *;
using Base::Base;
/// Merge every cell's value of current map into the destination map.
/// Func should have signature void(Mapped & dst, Mapped & src, bool emplaced).
/// Each filled cell in current map will invoke func once. If that map doesn't
/// have a key equals to the given cell, a new cell gets emplaced into that map,
/// and func is invoked with the third argument emplaced set to true. Otherwise
/// emplaced is set to false.
template <typename Func>
void ALWAYS_INLINE mergeToViaEmplace(Self & that, Func && func)
{
if (this->m0.hasZero())
{
const bool emplace_new_zero = !that.m0.hasZero();
if (emplace_new_zero)
{
that.m0.setHasZero();
}
func(that.m0.zeroValue()->getSecond(), this->m0.zeroValue()->getSecond(),
emplace_new_zero);
}
this->m1.mergeToViaEmplace(that.m1, func);
this->m2.mergeToViaEmplace(that.m2, func);
this->m3.mergeToViaEmplace(that.m3, func);
this->ms.mergeToViaEmplace(that.ms, func);
}
/// Merge every cell's value of current map into the destination map via find.
/// Func should have signature void(Mapped & dst, Mapped & src, bool exist).
/// Each filled cell in current map will invoke func once. If that map doesn't
/// have a key equals to the given cell, func is invoked with the third argument
/// exist set to false. Otherwise exist is set to true.
template <typename Func>
void ALWAYS_INLINE mergeToViaFind(Self & that, Func && func)
{
if (this->m0.hasZero())
{
if (that.m0.hasZero())
{
func(that.m0.zeroValue()->getSecond(), this->m0.zeroValue()->getSecond(), true);
}
else
{
func(this->m0.zeroValue()->getSecond(), this->m0.zeroValue()->getSecond(), false);
}
}
this->m1.mergeToViaFind(that.m1, func);
this->m2.mergeToViaFind(that.m2, func);
this->m3.mergeToViaFind(that.m3, func);
this->ms.mergeToViaFind(that.ms, func);
}
mapped_type & ALWAYS_INLINE operator[](Key x)
{
bool inserted;
LookupResult it = nullptr;
emplace(x, it, inserted);
if (inserted)
new (it) mapped_type();
return *it;
}
template <typename Func>
void ALWAYS_INLINE forEachValue(Func && func)
{
if (this->m0.size())
{
func(StringRef{}, this->m0.zeroValue()->getSecond());
}
for (auto & v : this->m1)
{
func(toStringRef(v.getFirst()), v.getSecond());
}
for (auto & v : this->m2)
{
func(toStringRef(v.getFirst()), v.getSecond());
}
for (auto & v : this->m3)
{
func(toStringRef(v.getFirst()), v.getSecond());
}
for (auto & v : this->ms)
{
func(v.getFirst(), v.getSecond());
}
}
template <typename Func>
void ALWAYS_INLINE forEachMapped(Func && func)
{
if (this->m0.size())
func(this->m0.zeroValue()->getSecond());
for (auto & v : this->m1)
func(v.getSecond());
for (auto & v : this->m2)
func(v.getSecond());
for (auto & v : this->m3)
func(v.getSecond());
for (auto & v : this->ms)
func(v.getSecond());
}
};

View File

@ -0,0 +1,371 @@
#pragma once
#include <Common/HashTable/HashMap.h>
#include <Common/HashTable/HashTable.h>
struct StringKey0
{
};
using StringKey8 = UInt64;
using StringKey16 = DB::UInt128;
struct StringKey24
{
UInt64 a;
UInt64 b;
UInt64 c;
bool operator==(const StringKey24 rhs) const { return a == rhs.a && b == rhs.b && c == rhs.c; }
};
inline StringRef ALWAYS_INLINE toStringRef(const StringKey8 & n)
{
return {reinterpret_cast<const char *>(&n), 8ul - (__builtin_clzll(n) >> 3)};
}
inline StringRef ALWAYS_INLINE toStringRef(const StringKey16 & n)
{
return {reinterpret_cast<const char *>(&n), 16ul - (__builtin_clzll(n.high) >> 3)};
}
inline StringRef ALWAYS_INLINE toStringRef(const StringKey24 & n)
{
return {reinterpret_cast<const char *>(&n), 24ul - (__builtin_clzll(n.c) >> 3)};
}
struct StringHashTableHash
{
#if defined(__SSE4_2__)
size_t ALWAYS_INLINE operator()(StringKey8 key) const
{
size_t res = -1ULL;
res = _mm_crc32_u64(res, key);
return res;
}
size_t ALWAYS_INLINE operator()(StringKey16 key) const
{
size_t res = -1ULL;
res = _mm_crc32_u64(res, key.low);
res = _mm_crc32_u64(res, key.high);
return res;
}
size_t ALWAYS_INLINE operator()(StringKey24 key) const
{
size_t res = -1ULL;
res = _mm_crc32_u64(res, key.a);
res = _mm_crc32_u64(res, key.b);
res = _mm_crc32_u64(res, key.c);
return res;
}
#else
size_t ALWAYS_INLINE operator()(StringKey8 key) const
{
return CityHash_v1_0_2::CityHash64(reinterpret_cast<const char *>(&key), 8);
}
size_t ALWAYS_INLINE operator()(StringKey16 key) const
{
return CityHash_v1_0_2::CityHash64(reinterpret_cast<const char *>(&key), 16);
}
size_t ALWAYS_INLINE operator()(StringKey24 key) const
{
return CityHash_v1_0_2::CityHash64(reinterpret_cast<const char *>(&key), 24);
}
#endif
size_t ALWAYS_INLINE operator()(StringRef key) const
{
return StringRefHash()(key);
}
};
template <typename Cell>
struct StringHashTableEmpty
{
using Self = StringHashTableEmpty;
bool has_zero = false;
std::aligned_storage_t<sizeof(Cell), alignof(Cell)> zero_value_storage; /// Storage of element with zero key.
public:
bool hasZero() const { return has_zero; }
void setHasZero()
{
has_zero = true;
new (zeroValue()) Cell();
}
void setHasZero(const Cell & other)
{
has_zero = true;
new (zeroValue()) Cell(other);
}
void clearHasZero()
{
has_zero = false;
if (!std::is_trivially_destructible_v<Cell>)
zeroValue()->~Cell();
}
Cell * zeroValue() { return reinterpret_cast<Cell *>(&zero_value_storage); }
const Cell * zeroValue() const { return reinterpret_cast<const Cell *>(&zero_value_storage); }
using LookupResult = Cell *;
using ConstLookupResult = const Cell *;
template <typename KeyHolder>
void ALWAYS_INLINE emplace(KeyHolder &&, LookupResult & it, bool & inserted, size_t /* hash */)
{
if (!hasZero())
{
setHasZero();
inserted = true;
}
else
inserted = false;
it = zeroValue();
}
template <typename Key>
LookupResult ALWAYS_INLINE find(Key, size_t /* hash */)
{
return hasZero() ? zeroValue() : nullptr;
}
void write(DB::WriteBuffer & wb) const { zeroValue()->write(wb); }
void writeText(DB::WriteBuffer & wb) const { zeroValue()->writeText(wb); }
void read(DB::ReadBuffer & rb) { zeroValue()->read(rb); }
void readText(DB::ReadBuffer & rb) { zeroValue()->readText(rb); }
size_t size() const { return hasZero() ? 1 : 0; }
bool empty() const { return !hasZero(); }
size_t getBufferSizeInBytes() const { return sizeof(Cell); }
size_t getCollisions() const { return 0; }
};
template <size_t initial_size_degree = 8>
struct StringHashTableGrower : public HashTableGrower<initial_size_degree>
{
// Smooth growing for string maps
void increaseSize() { this->size_degree += 1; }
};
template <typename SubMaps>
class StringHashTable : private boost::noncopyable
{
protected:
static constexpr size_t NUM_MAPS = 5;
// Map for storing empty string
using T0 = typename SubMaps::T0;
// Short strings are stored as numbers
using T1 = typename SubMaps::T1;
using T2 = typename SubMaps::T2;
using T3 = typename SubMaps::T3;
// Long strings are stored as StringRef along with saved hash
using Ts = typename SubMaps::Ts;
using Self = StringHashTable;
template <typename, typename, size_t>
friend class TwoLevelStringHashTable;
T0 m0;
T1 m1;
T2 m2;
T3 m3;
Ts ms;
public:
using Key = StringRef;
using key_type = Key;
using value_type = typename Ts::value_type;
using LookupResult = typename Ts::mapped_type *;
StringHashTable() {}
StringHashTable(size_t reserve_for_num_elements)
: m1{reserve_for_num_elements / 4}
, m2{reserve_for_num_elements / 4}
, m3{reserve_for_num_elements / 4}
, ms{reserve_for_num_elements / 4}
{
}
StringHashTable(StringHashTable && rhs) { *this = std::move(rhs); }
~StringHashTable() {}
public:
// Dispatch is written in a way that maximizes the performance:
// 1. Always memcpy 8 times bytes
// 2. Use switch case extension to generate fast dispatching table
// 3. Funcs are named callables that can be force_inlined
// NOTE: It relies on Little Endianness
template <typename KeyHolder, typename Func>
decltype(auto) ALWAYS_INLINE dispatch(KeyHolder && key_holder, Func && func)
{
const StringRef & x = keyHolderGetKey(key_holder);
const size_t sz = x.size;
if (sz == 0)
{
static constexpr StringKey0 key0{};
keyHolderDiscardKey(key_holder);
return func(m0, key0, 0);
}
const char * p = x.data;
// pending bits that needs to be shifted out
const char s = (-sz & 7) * 8;
union
{
StringKey8 k8;
StringKey16 k16;
StringKey24 k24;
UInt64 n[3];
};
StringHashTableHash hash;
switch ((sz - 1) >> 3)
{
case 0: // 1..8 bytes
{
// first half page
if ((reinterpret_cast<uintptr_t>(p) & 2048) == 0)
{
memcpy(&n[0], p, 8);
n[0] &= -1ul >> s;
}
else
{
const char * lp = x.data + x.size - 8;
memcpy(&n[0], lp, 8);
n[0] >>= s;
}
keyHolderDiscardKey(key_holder);
return func(m1, k8, hash(k8));
}
case 1: // 9..16 bytes
{
memcpy(&n[0], p, 8);
const char * lp = x.data + x.size - 8;
memcpy(&n[1], lp, 8);
n[1] >>= s;
keyHolderDiscardKey(key_holder);
return func(m2, k16, hash(k16));
}
case 2: // 17..24 bytes
{
memcpy(&n[0], p, 16);
const char * lp = x.data + x.size - 8;
memcpy(&n[2], lp, 8);
n[2] >>= s;
keyHolderDiscardKey(key_holder);
return func(m3, k24, hash(k24));
}
default: // >= 25 bytes
{
return func(ms, std::forward<KeyHolder>(key_holder), hash(x));
}
}
}
struct EmplaceCallable
{
LookupResult & mapped;
bool & inserted;
EmplaceCallable(LookupResult & mapped_, bool & inserted_)
: mapped(mapped_), inserted(inserted_) {}
template <typename Map, typename KeyHolder>
void ALWAYS_INLINE operator()(Map & map, KeyHolder && key_holder, size_t hash)
{
typename Map::LookupResult result;
map.emplace(key_holder, result, inserted, hash);
mapped = lookupResultGetMapped(result);
}
};
template <typename KeyHolder>
void ALWAYS_INLINE emplace(KeyHolder && key_holder, LookupResult & it, bool & inserted)
{
this->dispatch(key_holder, EmplaceCallable(it, inserted));
}
struct FindCallable
{
// find() doesn't need any key memory management, so we don't work with
// any key holders here, only with normal keys. The key type is still
// different for every subtable, this is why it is a template parameter.
template <typename Submap, typename SubmapKey>
LookupResult ALWAYS_INLINE operator()(Submap & map, const SubmapKey & key, size_t hash)
{
return lookupResultGetMapped(map.find(key, hash));
}
};
LookupResult ALWAYS_INLINE find(Key x)
{
return dispatch(x, FindCallable{});
}
void write(DB::WriteBuffer & wb) const
{
m0.write(wb);
m1.write(wb);
m2.write(wb);
m3.write(wb);
ms.write(wb);
}
void writeText(DB::WriteBuffer & wb) const
{
m0.writeText(wb);
DB::writeChar(',', wb);
m1.writeText(wb);
DB::writeChar(',', wb);
m2.writeText(wb);
DB::writeChar(',', wb);
m3.writeText(wb);
DB::writeChar(',', wb);
ms.writeText(wb);
}
void read(DB::ReadBuffer & rb)
{
m0.read(rb);
m1.read(rb);
m2.read(rb);
m3.read(rb);
ms.read(rb);
}
void readText(DB::ReadBuffer & rb)
{
m0.readText(rb);
DB::assertChar(',', rb);
m1.readText(rb);
DB::assertChar(',', rb);
m2.readText(rb);
DB::assertChar(',', rb);
m3.readText(rb);
DB::assertChar(',', rb);
ms.readText(rb);
}
size_t size() const { return m0.size() + m1.size() + m2.size() + m3.size() + ms.size(); }
bool empty() const { return m0.empty() && m1.empty() && m2.empty() && m3.empty() && ms.empty(); }
size_t getBufferSizeInBytes() const
{
return m0.getBufferSizeInBytes() + m1.getBufferSizeInBytes() + m2.getBufferSizeInBytes() + m3.getBufferSizeInBytes()
+ ms.getBufferSizeInBytes();
}
void clearAndShrink()
{
m1.clearHasZero();
m1.clearAndShrink();
m2.clearAndShrink();
m3.clearAndShrink();
ms.clearAndShrink();
}
};

View File

@ -0,0 +1,37 @@
#pragma once
#include <Common/HashTable/StringHashMap.h>
#include <Common/HashTable/TwoLevelStringHashTable.h>
template <typename TMapped, typename Allocator = HashTableAllocator, template <typename...> typename ImplTable = StringHashMap>
class TwoLevelStringHashMap : public TwoLevelStringHashTable<StringHashMapSubMaps<TMapped, Allocator>, ImplTable<TMapped, Allocator>>
{
public:
using Key = StringRef;
using key_type = Key;
using Self = TwoLevelStringHashMap;
using Base = TwoLevelStringHashTable<StringHashMapSubMaps<TMapped, Allocator>, StringHashMap<TMapped, Allocator>>;
using Base::Base;
using typename Base::Impl;
using mapped_type = TMapped;
using value_type = typename Base::value_type;
using LookupResult = typename Base::LookupResult;
template <typename Func>
void ALWAYS_INLINE forEachMapped(Func && func)
{
for (auto i = 0u; i < this->NUM_BUCKETS; ++i)
return this->impls[i].forEachMapped(func);
}
mapped_type & ALWAYS_INLINE operator[](Key x)
{
bool inserted;
LookupResult it;
emplace(x, it, inserted);
if (inserted)
new (lookupResultGetMapped(it)) mapped_type();
return *lookupResultGetMapped(it);
}
};

View File

@ -0,0 +1,218 @@
#pragma once
#include <Common/HashTable/StringHashTable.h>
template <typename SubMaps, typename ImplTable = StringHashTable<SubMaps>, size_t BITS_FOR_BUCKET = 8>
class TwoLevelStringHashTable : private boost::noncopyable
{
protected:
using HashValue = size_t;
using Self = TwoLevelStringHashTable;
public:
using Key = StringRef;
using Impl = ImplTable;
static constexpr size_t NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET;
static constexpr size_t MAX_BUCKET = NUM_BUCKETS - 1;
// TODO: currently hashing contains redundant computations when doing distributed or external aggregations
size_t hash(const Key & x) const
{
return const_cast<Self &>(*this).dispatch(x,
[&](const auto &, const auto &, size_t hash) { return hash; });
}
size_t operator()(const Key & x) const { return hash(x); }
/// NOTE Bad for hash tables with more than 2^32 cells.
static size_t getBucketFromHash(size_t hash_value) { return (hash_value >> (32 - BITS_FOR_BUCKET)) & MAX_BUCKET; }
public:
using key_type = typename Impl::key_type;
using value_type = typename Impl::value_type;
using LookupResult = typename Impl::LookupResult;
Impl impls[NUM_BUCKETS];
TwoLevelStringHashTable() {}
template <typename Source>
TwoLevelStringHashTable(const Source & src)
{
if (src.m0.hasZero())
impls[0].m0.setHasZero(*src.m0.zeroValue());
for (auto & v : src.m1)
{
size_t hash_value = v.getHash(src.m1);
size_t buck = getBucketFromHash(hash_value);
impls[buck].m1.insertUniqueNonZero(&v, hash_value);
}
for (auto & v : src.m2)
{
size_t hash_value = v.getHash(src.m2);
size_t buck = getBucketFromHash(hash_value);
impls[buck].m2.insertUniqueNonZero(&v, hash_value);
}
for (auto & v : src.m3)
{
size_t hash_value = v.getHash(src.m3);
size_t buck = getBucketFromHash(hash_value);
impls[buck].m3.insertUniqueNonZero(&v, hash_value);
}
for (auto & v : src.ms)
{
size_t hash_value = v.getHash(src.ms);
size_t buck = getBucketFromHash(hash_value);
impls[buck].ms.insertUniqueNonZero(&v, hash_value);
}
}
// This function is mostly the same as StringHashTable::dispatch, but with
// added bucket computation. See the comments there.
template <typename Func, typename KeyHolder>
decltype(auto) ALWAYS_INLINE dispatch(KeyHolder && key_holder, Func && func)
{
const StringRef & x = keyHolderGetKey(key_holder);
const size_t sz = x.size;
if (sz == 0)
{
static constexpr StringKey0 key0{};
keyHolderDiscardKey(key_holder);
return func(impls[0].m0, key0, 0);
}
const char * p = x.data;
// pending bits that needs to be shifted out
const char s = (-sz & 7) * 8;
union
{
StringKey8 k8;
StringKey16 k16;
StringKey24 k24;
UInt64 n[3];
};
StringHashTableHash hash;
switch ((sz - 1) >> 3)
{
case 0:
{
// first half page
if ((reinterpret_cast<uintptr_t>(p) & 2048) == 0)
{
memcpy(&n[0], p, 8);
n[0] &= -1ul >> s;
}
else
{
const char * lp = x.data + x.size - 8;
memcpy(&n[0], lp, 8);
n[0] >>= s;
}
auto res = hash(k8);
auto buck = getBucketFromHash(res);
keyHolderDiscardKey(key_holder);
return func(impls[buck].m1, k8, res);
}
case 1:
{
memcpy(&n[0], p, 8);
const char * lp = x.data + x.size - 8;
memcpy(&n[1], lp, 8);
n[1] >>= s;
auto res = hash(k16);
auto buck = getBucketFromHash(res);
keyHolderDiscardKey(key_holder);
return func(impls[buck].m2, k16, res);
}
case 2:
{
memcpy(&n[0], p, 16);
const char * lp = x.data + x.size - 8;
memcpy(&n[2], lp, 8);
n[2] >>= s;
auto res = hash(k24);
auto buck = getBucketFromHash(res);
keyHolderDiscardKey(key_holder);
return func(impls[buck].m3, k24, res);
}
default:
{
auto res = hash(x);
auto buck = getBucketFromHash(res);
return func(impls[buck].ms, std::forward<KeyHolder>(key_holder), res);
}
}
}
template <typename KeyHolder>
void ALWAYS_INLINE emplace(KeyHolder && key_holder, LookupResult & it, bool & inserted)
{
dispatch(key_holder, typename Impl::EmplaceCallable{it, inserted});
}
LookupResult ALWAYS_INLINE find(Key x)
{
return dispatch(x, typename Impl::FindCallable{});
}
void write(DB::WriteBuffer & wb) const
{
for (size_t i = 0; i < NUM_BUCKETS; ++i)
impls[i].write(wb);
}
void writeText(DB::WriteBuffer & wb) const
{
for (size_t i = 0; i < NUM_BUCKETS; ++i)
{
if (i != 0)
DB::writeChar(',', wb);
impls[i].writeText(wb);
}
}
void read(DB::ReadBuffer & rb)
{
for (size_t i = 0; i < NUM_BUCKETS; ++i)
impls[i].read(rb);
}
void readText(DB::ReadBuffer & rb)
{
for (size_t i = 0; i < NUM_BUCKETS; ++i)
{
if (i != 0)
DB::assertChar(',', rb);
impls[i].readText(rb);
}
}
size_t size() const
{
size_t res = 0;
for (size_t i = 0; i < NUM_BUCKETS; ++i)
res += impls[i].size();
return res;
}
bool empty() const
{
for (size_t i = 0; i < NUM_BUCKETS; ++i)
if (!impls[i].empty())
return false;
return true;
}
size_t getBufferSizeInBytes() const
{
size_t res = 0;
for (size_t i = 0; i < NUM_BUCKETS; ++i)
res += impls[i].getBufferSizeInBytes();
return res;
}
};

View File

@ -112,7 +112,8 @@ namespace Protocol
Cancel = 3, /// Cancel the query execution.
Ping = 4, /// Check that connection to the server is alive.
TablesStatusRequest = 5, /// Check status of tables on the server.
KeepAlive = 6 /// Keep the connection alive
KeepAlive = 6, /// Keep the connection alive
Scalar = 7 /// A block of data (compressed or not).
};
inline const char * toString(UInt64 packet)

View File

@ -379,6 +379,8 @@ struct Settings : public SettingsCollection<Settings>
M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.") \
M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.") \
\
M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.") \
\
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
\
M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \

View File

@ -6,11 +6,18 @@
namespace DB
{
// Used to disable implicit casting for certain overloaded types such as Field, which leads to
// overload resolution ambiguity.
template <typename T> struct Dumpable;
template <typename T>
std::ostream & operator<<(std::ostream & stream, const typename Dumpable<T>::Type & what);
class IBlockInputStream;
std::ostream & operator<<(std::ostream & stream, const IBlockInputStream & what);
class Field;
std::ostream & operator<<(std::ostream & stream, const Field & what);
template <> struct Dumpable<Field> { using Type = Field; };
struct NameAndTypePair;
std::ostream & operator<<(std::ostream & stream, const NameAndTypePair & what);

View File

@ -23,8 +23,8 @@ namespace ErrorCodes
RemoteBlockInputStream::RemoteBlockInputStream(
Connection & connection,
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
{
if (settings)
context.setSettings(*settings);
@ -38,8 +38,8 @@ RemoteBlockInputStream::RemoteBlockInputStream(
RemoteBlockInputStream::RemoteBlockInputStream(
std::vector<IConnectionPool::Entry> && connections,
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
{
if (settings)
context.setSettings(*settings);
@ -54,8 +54,8 @@ RemoteBlockInputStream::RemoteBlockInputStream(
RemoteBlockInputStream::RemoteBlockInputStream(
const ConnectionPoolWithFailoverPtr & pool,
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
{
if (settings)
context.setSettings(*settings);
@ -120,6 +120,11 @@ void RemoteBlockInputStream::cancel(bool kill)
tryCancel("Cancelling query");
}
void RemoteBlockInputStream::sendScalars()
{
multiplexed_connections->sendScalarsData(scalars);
}
void RemoteBlockInputStream::sendExternalTables()
{
size_t count = multiplexed_connections->size();
@ -308,6 +313,8 @@ void RemoteBlockInputStream::sendQuery()
established = false;
sent_query = true;
if (settings.enable_scalar_subquery_optimization)
sendScalars();
sendExternalTables();
}

View File

@ -25,7 +25,7 @@ public:
RemoteBlockInputStream(
Connection & connection,
const String & query_, const Block & header_, const Context & context_, const Settings * settings = nullptr,
const ThrottlerPtr & throttler = nullptr, const Tables & external_tables_ = Tables(),
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
/// Accepts several connections already taken from pool.
@ -33,7 +33,7 @@ public:
RemoteBlockInputStream(
std::vector<IConnectionPool::Entry> && connections,
const String & query_, const Block & header_, const Context & context_, const Settings * settings = nullptr,
const ThrottlerPtr & throttler = nullptr, const Tables & external_tables_ = Tables(),
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
/// Takes a pool and gets one or several connections from it.
@ -41,7 +41,7 @@ public:
RemoteBlockInputStream(
const ConnectionPoolWithFailoverPtr & pool,
const String & query_, const Block & header_, const Context & context_, const Settings * settings = nullptr,
const ThrottlerPtr & throttler = nullptr, const Tables & external_tables_ = Tables(),
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
~RemoteBlockInputStream() override;
@ -71,6 +71,9 @@ public:
Block getHeader() const override { return header; }
protected:
/// Send all scalars to remote servers
void sendScalars();
/// Send all temporary tables to remote servers
void sendExternalTables();
@ -103,6 +106,8 @@ private:
String query_id = "";
Context context;
/// Scalars needed to be sent to remote servers
Scalars scalars;
/// Temporary tables needed to be sent to remote servers
Tables external_tables;
QueryProcessingStage::Enum stage;

View File

@ -8,6 +8,7 @@ namespace DB
void registerFunctionsJSON(FunctionFactory & factory)
{
factory.registerFunction<FunctionJSON<NameJSONHas, JSONHasImpl>>();
factory.registerFunction<FunctionJSON<NameIsValidJSON, IsValidJSONImpl>>();
factory.registerFunction<FunctionJSON<NameJSONLength, JSONLengthImpl>>();
factory.registerFunction<FunctionJSON<NameJSONKey, JSONKeyImpl>>();
factory.registerFunction<FunctionJSON<NameJSONType, JSONTypeImpl>>();

View File

@ -279,6 +279,7 @@ private:
struct NameJSONHas { static constexpr auto name{"JSONHas"}; };
struct NameIsValidJSON { static constexpr auto name{"isValidJSON"}; };
struct NameJSONLength { static constexpr auto name{"JSONLength"}; };
struct NameJSONKey { static constexpr auto name{"JSONKey"}; };
struct NameJSONType { static constexpr auto name{"JSONType"}; };
@ -311,6 +312,36 @@ public:
};
template <typename JSONParser>
class IsValidJSONImpl
{
public:
static DataTypePtr getType(const char * function_name, const ColumnsWithTypeAndName & arguments)
{
if (arguments.size() != 1)
{
/// IsValidJSON() shouldn't get parameters other than JSON.
throw Exception{"Function " + String(function_name) + " needs exactly one argument",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
}
return std::make_shared<DataTypeUInt8>();
}
using Iterator = typename JSONParser::Iterator;
static bool addValueToColumn(IColumn & dest, const Iterator &)
{
/// This function is called only if JSON is valid.
/// If JSON isn't valid then `FunctionJSON::Executor::run()` adds default value (=zero) to `dest` without calling this function.
ColumnVector<UInt8> & col_vec = assert_cast<ColumnVector<UInt8> &>(dest);
col_vec.insertValue(1);
return true;
}
static constexpr size_t num_extra_arguments = 0;
static void prepare(const char *, const Block &, const ColumnNumbers &, size_t) {}
};
template <typename JSONParser>
class JSONLengthImpl
{

View File

@ -0,0 +1,68 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Interpreters/Context.h>
#include <Common/Macros.h>
#include <Core/Field.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/** Get scalar value of sub queries from query context via IAST::Hash.
*/
class FunctionGetScalar : public IFunction
{
public:
static constexpr auto name = "__getScalar";
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionGetScalar>(context);
}
FunctionGetScalar(const Context & context_) : context(context_) {}
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override
{
return 1;
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 1 || !isString(arguments[0].type) || !isColumnConst(*arguments[0].column))
throw Exception("Function " + getName() + " accepts one const string argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto scalar_name = assert_cast<const ColumnConst &>(*arguments[0].column).getField().get<String>();
scalar = context.getScalar(scalar_name).getByPosition(0);
return scalar.type;
}
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = ColumnConst::create(scalar.column, input_rows_count);
}
private:
mutable ColumnWithTypeAndName scalar;
const Context & context;
};
void registerFunctionGetScalar(FunctionFactory & factory)
{
factory.registerFunction<FunctionGetScalar>();
}
}

View File

@ -52,6 +52,7 @@ void registerFunctionEvalMLMethod(FunctionFactory &);
void registerFunctionBasename(FunctionFactory &);
void registerFunctionTransform(FunctionFactory &);
void registerFunctionGetMacro(FunctionFactory &);
void registerFunctionGetScalar(FunctionFactory &);
#if USE_ICU
void registerFunctionConvertCharset(FunctionFactory &);
@ -106,6 +107,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
registerFunctionBasename(factory);
registerFunctionTransform(factory);
registerFunctionGetMacro(factory);
registerFunctionGetScalar(factory);
#if USE_ICU
registerFunctionConvertCharset(factory);

View File

@ -11,6 +11,9 @@
#include <Common/HashTable/FixedHashMap.h>
#include <Common/HashTable/HashMap.h>
#include <Common/HashTable/TwoLevelHashMap.h>
#include <Common/HashTable/StringHashMap.h>
#include <Common/HashTable/TwoLevelStringHashMap.h>
#include <Common/ThreadPool.h>
#include <Common/UInt128.h>
#include <Common/LRUCache.h>
@ -69,12 +72,20 @@ using AggregatedDataWithUInt8Key = FixedHashMap<UInt8, AggregateDataPtr>;
using AggregatedDataWithUInt16Key = FixedHashMap<UInt16, AggregateDataPtr>;
using AggregatedDataWithUInt64Key = HashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>>;
using AggregatedDataWithShortStringKey = StringHashMap<AggregateDataPtr>;
using AggregatedDataWithStringKey = HashMapWithSavedHash<StringRef, AggregateDataPtr>;
using AggregatedDataWithKeys128 = HashMap<UInt128, AggregateDataPtr, UInt128HashCRC32>;
using AggregatedDataWithKeys256 = HashMap<UInt256, AggregateDataPtr, UInt256HashCRC32>;
using AggregatedDataWithUInt64KeyTwoLevel = TwoLevelHashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>>;
using AggregatedDataWithShortStringKeyTwoLevel = TwoLevelStringHashMap<AggregateDataPtr>;
using AggregatedDataWithStringKeyTwoLevel = TwoLevelHashMapWithSavedHash<StringRef, AggregateDataPtr>;
using AggregatedDataWithKeys128TwoLevel = TwoLevelHashMap<UInt128, AggregateDataPtr, UInt128HashCRC32>;
using AggregatedDataWithKeys256TwoLevel = TwoLevelHashMap<UInt256, AggregateDataPtr, UInt256HashCRC32>;
@ -139,6 +150,8 @@ struct AggregationDataWithNullKeyTwoLevel : public Base
template <typename ... Types>
using HashTableWithNullKey = AggregationDataWithNullKey<HashMapTable<Types ...>>;
template <typename ... Types>
using StringHashTableWithNullKey = AggregationDataWithNullKey<StringHashMap<Types ...>>;
using AggregatedDataWithNullableUInt8Key = AggregationDataWithNullKey<AggregatedDataWithUInt8Key>;
using AggregatedDataWithNullableUInt16Key = AggregationDataWithNullKey<AggregatedDataWithUInt16Key>;
@ -149,6 +162,10 @@ using AggregatedDataWithNullableStringKey = AggregationDataWithNullKey<Aggregate
using AggregatedDataWithNullableUInt64KeyTwoLevel = AggregationDataWithNullKeyTwoLevel<
TwoLevelHashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>,
TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>;
using AggregatedDataWithNullableShortStringKeyTwoLevel = AggregationDataWithNullKeyTwoLevel<
TwoLevelStringHashMap<AggregateDataPtr, HashTableAllocator, StringHashTableWithNullKey>>;
using AggregatedDataWithNullableStringKeyTwoLevel = AggregationDataWithNullKeyTwoLevel<
TwoLevelHashMapWithSavedHash<StringRef, AggregateDataPtr, DefaultHash<StringRef>,
TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>;
@ -216,6 +233,32 @@ struct AggregationMethodString
};
/// Same as above but without cache
template <typename TData>
struct AggregationMethodStringNoCache
{
using Data = TData;
using Key = typename Data::key_type;
using Mapped = typename Data::mapped_type;
Data data;
AggregationMethodStringNoCache() {}
template <typename Other>
AggregationMethodStringNoCache(const Other & other) : data(other.data) {}
using State = ColumnsHashing::HashMethodString<typename Data::value_type, Mapped, true, false>;
static const bool low_cardinality_optimization = false;
static void insertKeyIntoColumns(const StringRef & key, MutableColumns & key_columns, const Sizes &)
{
key_columns[0]->insertData(key.data, key.size);
}
};
/// For the case where there is one fixed-length string key.
template <typename TData>
struct AggregationMethodFixedString
@ -241,6 +284,32 @@ struct AggregationMethodFixedString
}
};
/// Same as above but without cache
template <typename TData>
struct AggregationMethodFixedStringNoCache
{
using Data = TData;
using Key = typename Data::key_type;
using Mapped = typename Data::mapped_type;
Data data;
AggregationMethodFixedStringNoCache() {}
template <typename Other>
AggregationMethodFixedStringNoCache(const Other & other) : data(other.data) {}
using State = ColumnsHashing::HashMethodFixedString<typename Data::value_type, Mapped, true, false>;
static const bool low_cardinality_optimization = false;
static void insertKeyIntoColumns(const StringRef & key, MutableColumns & key_columns, const Sizes &)
{
key_columns[0]->insertData(key.data, key.size);
}
};
/// Single low cardinality column.
template <typename SingleColumnMethod>
struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod
@ -434,16 +503,16 @@ struct AggregatedDataVariants : private boost::noncopyable
std::unique_ptr<AggregationMethodOneNumber<UInt32, AggregatedDataWithUInt64Key>> key32;
std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64Key>> key64;
std::unique_ptr<AggregationMethodString<AggregatedDataWithStringKey>> key_string;
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKey>> key_fixed_string;
std::unique_ptr<AggregationMethodStringNoCache<AggregatedDataWithShortStringKey>> key_string;
std::unique_ptr<AggregationMethodFixedStringNoCache<AggregatedDataWithShortStringKey>> key_fixed_string;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128>> keys128;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256>> keys256;
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKey>> serialized;
std::unique_ptr<AggregationMethodOneNumber<UInt32, AggregatedDataWithUInt64KeyTwoLevel>> key32_two_level;
std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64KeyTwoLevel>> key64_two_level;
std::unique_ptr<AggregationMethodString<AggregatedDataWithStringKeyTwoLevel>> key_string_two_level;
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKeyTwoLevel>> key_fixed_string_two_level;
std::unique_ptr<AggregationMethodStringNoCache<AggregatedDataWithShortStringKeyTwoLevel>> key_string_two_level;
std::unique_ptr<AggregationMethodFixedStringNoCache<AggregatedDataWithShortStringKeyTwoLevel>> key_fixed_string_two_level;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128TwoLevel>> keys128_two_level;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256TwoLevel>> keys256_two_level;
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKeyTwoLevel>> serialized_two_level;

View File

@ -33,11 +33,13 @@ SelectStreamFactory::SelectStreamFactory(
const Block & header_,
QueryProcessingStage::Enum processed_stage_,
QualifiedTableName main_table_,
const Scalars & scalars_,
const Tables & external_tables_)
: header(header_),
processed_stage{processed_stage_},
main_table(std::move(main_table_)),
table_func_ptr{nullptr},
scalars{scalars_},
external_tables{external_tables_}
{
}
@ -46,10 +48,12 @@ SelectStreamFactory::SelectStreamFactory(
const Block & header_,
QueryProcessingStage::Enum processed_stage_,
ASTPtr table_func_ptr_,
const Scalars & scalars_,
const Tables & external_tables_)
: header(header_),
processed_stage{processed_stage_},
table_func_ptr{table_func_ptr_},
scalars{scalars_},
external_tables{external_tables_}
{
}
@ -92,7 +96,8 @@ void SelectStreamFactory::createForShard(
auto emplace_remote_stream = [&]()
{
auto stream = std::make_shared<RemoteBlockInputStream>(shard_info.pool, query, header, context, nullptr, throttler, external_tables, processed_stage);
auto stream = std::make_shared<RemoteBlockInputStream>(
shard_info.pool, query, header, context, nullptr, throttler, scalars, external_tables, processed_stage);
stream->setPoolMode(PoolMode::GET_MANY);
if (!table_func_ptr)
stream->setMainTable(main_table);
@ -190,8 +195,8 @@ void SelectStreamFactory::createForShard(
auto lazily_create_stream = [
pool = shard_info.pool, shard_num = shard_info.shard_num, query, header = header, query_ast, context, throttler,
main_table = main_table, table_func_ptr = table_func_ptr, external_tables = external_tables, stage = processed_stage,
local_delay]()
main_table = main_table, table_func_ptr = table_func_ptr, scalars = scalars, external_tables = external_tables,
stage = processed_stage, local_delay]()
-> BlockInputStreamPtr
{
auto current_settings = context.getSettingsRef();
@ -233,7 +238,7 @@ void SelectStreamFactory::createForShard(
connections.emplace_back(std::move(try_result.entry));
return std::make_shared<RemoteBlockInputStream>(
std::move(connections), query, header, context, nullptr, throttler, external_tables, stage);
std::move(connections), query, header, context, nullptr, throttler, scalars, external_tables, stage);
}
};

View File

@ -18,6 +18,7 @@ public:
const Block & header_,
QueryProcessingStage::Enum processed_stage_,
QualifiedTableName main_table_,
const Scalars & scalars_,
const Tables & external_tables);
/// TableFunction in a query.
@ -25,6 +26,7 @@ public:
const Block & header_,
QueryProcessingStage::Enum processed_stage_,
ASTPtr table_func_ptr_,
const Scalars & scalars_,
const Tables & external_tables_);
void createForShard(
@ -38,6 +40,7 @@ private:
QueryProcessingStage::Enum processed_stage;
QualifiedTableName main_table;
ASTPtr table_func_ptr;
Scalars scalars;
Tables external_tables;
};

View File

@ -89,6 +89,8 @@ namespace ErrorCodes
extern const int SESSION_IS_LOCKED;
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
extern const int LOGICAL_ERROR;
extern const int SCALAR_ALREADY_EXISTS;
extern const int UNKNOWN_SCALAR;
}
@ -857,6 +859,21 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const
}
const Scalars & Context::getScalars() const
{
return scalars;
}
const Block & Context::getScalar(const String & name) const
{
auto it = scalars.find(name);
if (scalars.end() == it)
throw Exception("Scalar " + backQuoteIfNeed(name) + " doesn't exist (internal bug)", ErrorCodes::UNKNOWN_SCALAR);
return it->second;
}
Tables Context::getExternalTables() const
{
auto lock = getLock();
@ -954,6 +971,19 @@ void Context::addExternalTable(const String & table_name, const StoragePtr & sto
external_tables[table_name] = std::pair(storage, ast);
}
void Context::addScalar(const String & name, const Block & block)
{
scalars[name] = block;
}
bool Context::hasScalar(const String & name) const
{
return scalars.count(name);
}
StoragePtr Context::tryRemoveExternalTable(const String & table_name)
{
TableAndCreateASTs::const_iterator it = external_tables.find(table_name);

View File

@ -105,6 +105,9 @@ using InputInitializer = std::function<void(Context &, const StoragePtr &)>;
/// Callback for reading blocks of data from client for function input()
using InputBlocksReader = std::function<Block(Context &)>;
/// Scalar results of sub queries
using Scalars = std::map<String, Block>;
/// An empty interface for an arbitrary object that may be attached by a shared pointer
/// to query context, when using ClickHouse as a library.
struct IHostContext
@ -144,6 +147,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.
Scalars scalars;
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;
@ -264,11 +268,15 @@ public:
void assertDatabaseDoesntExist(const String & database_name) const;
void checkDatabaseAccessRights(const std::string & database_name) const;
const Scalars & getScalars() const;
const Block & getScalar(const String & name) const;
Tables getExternalTables() const;
StoragePtr tryGetExternalTable(const String & table_name) const;
StoragePtr getTable(const String & database_name, const String & table_name) const;
StoragePtr tryGetTable(const String & database_name, const String & table_name) const;
void addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast = {});
void addScalar(const String & name, const Block & block);
bool hasScalar(const String & name) const;
StoragePtr tryRemoveExternalTable(const String & table_name);
StoragePtr executeTableFunction(const ASTPtr & table_expression);

View File

@ -94,7 +94,7 @@ public:
, ands_only(true)
{}
void visit(const ASTFunction & node, ASTPtr & ast)
void visit(const ASTFunction & node, const ASTPtr & ast)
{
if (!ands_only)
return;
@ -231,8 +231,8 @@ private:
}
};
using CheckExpressionMatcher = OneTypeMatcher<CheckExpressionVisitorData, false>;
using CheckExpressionVisitor = InDepthNodeVisitor<CheckExpressionMatcher, true>;
using CheckExpressionMatcher = ConstOneTypeMatcher<CheckExpressionVisitorData, false>;
using CheckExpressionVisitor = ConstInDepthNodeVisitor<CheckExpressionMatcher, true>;
bool getTables(ASTSelectQuery & select, std::vector<JoinedTable> & joined_tables, size_t & num_comma)
@ -314,7 +314,7 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da
return;
CheckExpressionVisitor::Data visitor_data{joined_tables};
CheckExpressionVisitor(visitor_data).visit(select.refWhere());
CheckExpressionVisitor(visitor_data).visit(select.where());
if (visitor_data.complex())
return;

View File

@ -12,8 +12,11 @@
#include <Interpreters/addTypeConversionToAST.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/materializeBlock.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnTuple.h>
namespace DB
{
@ -53,8 +56,29 @@ void ExecuteScalarSubqueriesMatcher::visit(ASTPtr & ast, Data & data)
visit(*t, ast, data);
}
/// Converting to literal values might take a fair amount of overhead when the value is large, (e.g.
/// Array, BitMap, etc.), This conversion is required for constant folding, index lookup, branch
/// elimination. However, these optimizations should never be related to large values, thus we
/// blacklist them here.
static bool worthConvertingToLiteral(const Block & scalar)
{
auto scalar_type_name = scalar.safeGetByPosition(0).type->getFamilyName();
std::set<String> useless_literal_types = {"Array", "Tuple", "AggregateFunction", "Function", "Set", "LowCardinality"};
return !useless_literal_types.count(scalar_type_name);
}
void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr & ast, Data & data)
{
auto hash = subquery.getTreeHash();
auto scalar_query_hash_str = toString(hash.first) + "_" + toString(hash.second);
Block scalar;
if (data.context.hasQueryContext() && data.context.getQueryContext().hasScalar(scalar_query_hash_str))
scalar = data.context.getQueryContext().getScalar(scalar_query_hash_str);
else if (data.scalars.count(scalar_query_hash_str))
scalar = data.scalars[scalar_query_hash_str];
else
{
Context subquery_context = data.context;
Settings subquery_settings = data.context.getSettings();
subquery_settings.max_result_rows = 1;
@ -90,32 +114,40 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
throw;
}
block = materializeBlock(block);
size_t columns = block.columns();
if (columns == 1)
scalar = block;
else
{
auto lit = std::make_unique<ASTLiteral>((*block.safeGetByPosition(0).column)[0]);
ColumnWithTypeAndName ctn;
ctn.type = std::make_shared<DataTypeTuple>(block.getDataTypes());
ctn.column = ColumnTuple::create(block.getColumns());
scalar.insert(ctn);
}
}
const Settings & settings = data.context.getSettingsRef();
// Always convert to literals when there is no query context.
if (!settings.enable_scalar_subquery_optimization || worthConvertingToLiteral(scalar) || !data.context.hasQueryContext())
{
auto lit = std::make_unique<ASTLiteral>((*scalar.safeGetByPosition(0).column)[0]);
lit->alias = subquery.alias;
lit->prefer_alias_to_column_name = subquery.prefer_alias_to_column_name;
ast = addTypeConversionToAST(std::move(lit), block.safeGetByPosition(0).type->getName());
ast = addTypeConversionToAST(std::move(lit), scalar.safeGetByPosition(0).type->getName());
}
else
{
auto tuple = std::make_shared<ASTFunction>();
tuple->alias = subquery.alias;
ast = tuple;
tuple->name = "tuple";
auto exp_list = std::make_shared<ASTExpressionList>();
tuple->arguments = exp_list;
tuple->children.push_back(tuple->arguments);
auto func = makeASTFunction("__getScalar", std::make_shared<ASTLiteral>(scalar_query_hash_str));
func->alias = subquery.alias;
func->prefer_alias_to_column_name = subquery.prefer_alias_to_column_name;
ast = std::move(func);
}
exp_list->children.resize(columns);
for (size_t i = 0; i < columns; ++i)
{
exp_list->children[i] = addTypeConversionToAST(
std::make_unique<ASTLiteral>((*block.safeGetByPosition(i).column)[0]),
block.safeGetByPosition(i).type->getName());
}
}
data.scalars[scalar_query_hash_str] = std::move(scalar);
}
void ExecuteScalarSubqueriesMatcher::visit(const ASTFunction & func, ASTPtr & ast, Data & data)

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/typeid_cast.h>
#include <Core/Block.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
@ -36,6 +37,7 @@ public:
{
const Context & context;
size_t subquery_depth;
Scalars & scalars;
};
static bool needChildVisit(ASTPtr & node, const ASTPtr &);

View File

@ -10,13 +10,13 @@ namespace DB
/// Visits AST tree in depth, call functions for nodes according to Matcher type data.
/// You need to define Data, visit() and needChildVisit() in Matcher class.
template <typename Matcher, bool _top_to_bottom, typename T>
class InDepthNodeVisitorTemplate
template <typename Matcher, bool _top_to_bottom, typename T = ASTPtr>
class InDepthNodeVisitor
{
public:
using Data = typename Matcher::Data;
InDepthNodeVisitorTemplate(Data & data_, std::ostream * ostr_ = nullptr)
InDepthNodeVisitor(Data & data_, std::ostream * ostr_ = nullptr)
: data(data_),
visit_depth(0),
ostr(ostr_)
@ -49,42 +49,26 @@ private:
};
template <typename Matcher, bool top_to_bottom>
using InDepthNodeVisitor = InDepthNodeVisitorTemplate<Matcher, top_to_bottom, ASTPtr>;
template <typename Matcher, bool top_to_bottom>
using ConstInDepthNodeVisitor = InDepthNodeVisitorTemplate<Matcher, top_to_bottom, const ASTPtr>;
using ConstInDepthNodeVisitor = InDepthNodeVisitor<Matcher, top_to_bottom, const ASTPtr>;
/// Simple matcher for one node type without complex traversal logic.
template <typename _Data, bool _visit_children = true>
template <typename Data_, bool visit_children = true, typename T = ASTPtr>
class OneTypeMatcher
{
public:
using Data = _Data;
using Data = Data_;
using TypeToVisit = typename Data::TypeToVisit;
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return _visit_children; }
static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return visit_children; }
static void visit(ASTPtr & ast, Data & data)
static void visit(T & ast, Data & data)
{
if (auto * t = typeid_cast<TypeToVisit *>(ast.get()))
data.visit(*t, ast);
}
};
/// Links two simple matches into resulting one. There's no complex traversal logic: all the children would be visited.
template <typename First, typename Second>
class LinkedMatcher
{
public:
using Data = std::pair<typename First::Data, typename Second::Data>;
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; }
static void visit(ASTPtr & ast, Data & data)
{
First::visit(ast, data.first);
Second::visit(ast, data.second);
}
};
template <typename Data, bool visit_children = true>
using ConstOneTypeMatcher = OneTypeMatcher<Data, visit_children, const ASTPtr>;
}

View File

@ -1,29 +1,25 @@
#include <IO/ConcatReadBuffer.h>
#include <IO/ReadBufferFromMemory.h>
#include <Common/typeid_cast.h>
#include <Common/checkStackSize.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <DataStreams/AddingDefaultBlockOutputStream.h>
#include <DataStreams/AddingDefaultsBlockInputStream.h>
#include <DataStreams/CheckConstraintsBlockOutputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <DataStreams/ConvertingBlockInputStream.h>
#include <DataStreams/CountingBlockOutputStream.h>
#include <DataStreams/InputStreamFromASTInsertQuery.h>
#include <DataStreams/NullAndDoCopyBlockInputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <DataStreams/PushingToViewsBlockOutputStream.h>
#include <DataStreams/SquashingBlockOutputStream.h>
#include <DataStreams/InputStreamFromASTInsertQuery.h>
#include <DataStreams/copyData.h>
#include <IO/ConcatReadBuffer.h>
#include <IO/ReadBufferFromMemory.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Storages/Kafka/StorageKafka.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Parsers/ASTFunction.h>
#include <Common/checkStackSize.h>
namespace DB
@ -38,8 +34,12 @@ namespace ErrorCodes
InterpreterInsertQuery::InterpreterInsertQuery(
const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_, bool no_squash_)
: query_ptr(query_ptr_), context(context_), allow_materialized(allow_materialized_), no_squash(no_squash_)
const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_, bool no_squash_, bool no_destination_)
: query_ptr(query_ptr_)
, context(context_)
, allow_materialized(allow_materialized_)
, no_squash(no_squash_)
, no_destination(no_destination_)
{
checkStackSize();
}
@ -65,7 +65,7 @@ Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const
/// If the query does not include information about columns
if (!query.columns)
{
if (query.no_destination)
if (no_destination)
return table->getSampleBlockWithVirtuals();
else
return table_sample_non_materialized;
@ -102,7 +102,12 @@ BlockIO InterpreterInsertQuery::execute()
/// We create a pipeline of several streams, into which we will write data.
BlockOutputStreamPtr out;
out = std::make_shared<PushingToViewsBlockOutputStream>(query.database, query.table, table, context, query_ptr, query.no_destination);
/// NOTE: we explicitly ignore bound materialized views when inserting into Kafka Storage.
/// Otherwise we'll get duplicates when MV reads same rows again from Kafka.
if (table->noPushingToViews() && !no_destination)
out = table->write(query_ptr, context);
else
out = std::make_shared<PushingToViewsBlockOutputStream>(query.database, query.table, table, context, query_ptr, no_destination);
/// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side.
/// Client-side bufferization might cause excessive timeouts (especially in case of big blocks).

View File

@ -15,7 +15,12 @@ namespace DB
class InterpreterInsertQuery : public IInterpreter
{
public:
InterpreterInsertQuery(const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_ = false, bool no_squash_ = false);
InterpreterInsertQuery(
const ASTPtr & query_ptr_,
const Context & context_,
bool allow_materialized_ = false,
bool no_squash_ = false,
bool no_destination_ = false);
/** Prepare a request for execution. Return block streams
* - the stream into which you can write data to execute the query, if INSERT;
@ -35,6 +40,7 @@ private:
const Context & context;
const bool allow_materialized;
const bool no_squash;
const bool no_destination;
};

View File

@ -305,6 +305,12 @@ InterpreterSelectQuery::InterpreterSelectQuery(
syntax_analyzer_result = SyntaxAnalyzer(context, options).analyze(
query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, storage, NamesAndTypesList());
/// Save scalar sub queries's results in the query context
if (context.hasQueryContext())
for (const auto & it : syntax_analyzer_result->getScalars())
context.getQueryContext().addScalar(it.first, it.second);
query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
query_ptr, syntax_analyzer_result, context,
NameSet(required_result_column_names.begin(), required_result_column_names.end()),

View File

@ -37,8 +37,6 @@ namespace
class ExtractAsterisksMatcher
{
public:
using Visitor = InDepthNodeVisitor<ExtractAsterisksMatcher, true>;
struct Data
{
std::unordered_map<String, NamesAndTypesList> table_columns;
@ -76,30 +74,16 @@ public:
}
};
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return false; }
static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return false; }
static void visit(ASTPtr & ast, Data & data)
static void visit(const ASTPtr & ast, Data & data)
{
if (auto * t = ast->as<ASTSelectQuery>())
visit(*t, ast, data);
if (auto * t = ast->as<ASTExpressionList>())
visit(*t, ast, data);
}
private:
static void visit(ASTSelectQuery & node, ASTPtr &, Data & data)
{
if (data.table_columns.empty())
return;
Visitor(data).visit(node.refSelect());
if (!data.new_select_expression_list)
return;
node.setExpression(ASTSelectQuery::Expression::SELECT, std::move(data.new_select_expression_list));
}
static void visit(ASTExpressionList & node, ASTPtr &, Data & data)
static void visit(const ASTExpressionList & node, const ASTPtr &, Data & data)
{
bool has_asterisks = false;
data.new_select_expression_list = std::make_shared<ASTExpressionList>();
@ -375,7 +359,7 @@ using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>;
using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
using SetSubqueryAliasMatcher = OneTypeMatcher<SetSubqueryAliasVisitorData>;
using SetSubqueryAliasVisitor = InDepthNodeVisitor<SetSubqueryAliasMatcher, true>;
using ExtractAsterisksVisitor = ExtractAsterisksMatcher::Visitor;
using ExtractAsterisksVisitor = ConstInDepthNodeVisitor<ExtractAsterisksMatcher, true>;
using ColumnAliasesVisitor = ConstInDepthNodeVisitor<ColumnAliasesMatcher, true>;
using AppendSemanticMatcher = OneTypeMatcher<AppendSemanticVisitorData>;
using AppendSemanticVisitor = InDepthNodeVisitor<AppendSemanticMatcher, true>;
@ -389,7 +373,7 @@ void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data)
visit(*t, ast, data);
}
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data)
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & data)
{
using RevertedAliases = AsteriskSemantic::RevertedAliases;
@ -398,7 +382,12 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast
return;
ExtractAsterisksVisitor::Data asterisks_data(data.context, table_expressions);
ExtractAsterisksVisitor(asterisks_data).visit(ast);
if (!asterisks_data.table_columns.empty())
{
ExtractAsterisksVisitor(asterisks_data).visit(select.select());
if (asterisks_data.new_select_expression_list)
select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(asterisks_data.new_select_expression_list));
}
ColumnAliasesVisitor::Data aliases_data(getDatabaseAndTables(select, ""));
if (select.select())

View File

@ -220,10 +220,10 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query,
}
/// Replacing scalar subqueries with constant values.
void executeScalarSubqueries(ASTPtr & query, const Context & context, size_t subquery_depth)
void executeScalarSubqueries(ASTPtr & query, const Context & context, size_t subquery_depth, Scalars & scalars)
{
LogAST log;
ExecuteScalarSubqueriesVisitor::Data visitor_data{context, subquery_depth};
ExecuteScalarSubqueriesVisitor::Data visitor_data{context, subquery_depth, scalars};
ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query);
}
@ -871,7 +871,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
removeUnneededColumnsFromSelectClause(select_query, required_result_columns, remove_duplicates);
/// Executing scalar subqueries - replacing them with constant values.
executeScalarSubqueries(query, context, subquery_depth);
executeScalarSubqueries(query, context, subquery_depth, result.scalars);
/// Optimize if with constant condition after constants was substituted instead of scalar subqueries.
OptimizeIfWithConstantConditionVisitor(result.aliases).visit(query);

View File

@ -1,5 +1,6 @@
#pragma once
#include <Core/Block.h>
#include <Core/NamesAndTypes.h>
#include <Interpreters/Aliases.h>
#include <Interpreters/SelectQueryOptions.h>
@ -14,6 +15,7 @@ class ASTFunction;
class AnalyzedJoin;
class Context;
struct SelectQueryOptions;
using Scalars = std::map<String, Block>;
struct SyntaxAnalyzerResult
{
@ -43,8 +45,12 @@ struct SyntaxAnalyzerResult
/// Predicate optimizer overrides the sub queries
bool rewrite_subqueries = false;
/// Results of scalar sub queries
Scalars scalars;
void collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns);
Names requiredSourceColumns() const { return required_source_columns.getNames(); }
const Scalars & getScalars() const { return scalars; }
};
using SyntaxAnalyzerResultPtr = std::shared_ptr<const SyntaxAnalyzerResult>;

View File

@ -37,6 +37,12 @@ add_executable (hash_map_string_small hash_map_string_small.cpp)
target_include_directories (hash_map_string_small SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR})
target_link_libraries (hash_map_string_small PRIVATE dbms)
add_executable (string_hash_map string_hash_map.cpp)
target_link_libraries (string_hash_map PRIVATE dbms)
add_executable (string_hash_map_aggregation string_hash_map.cpp)
target_link_libraries (string_hash_map_aggregation PRIVATE dbms)
add_executable (two_level_hash_map two_level_hash_map.cpp)
target_include_directories (two_level_hash_map SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR})
target_link_libraries (two_level_hash_map PRIVATE dbms)

View File

@ -0,0 +1,246 @@
#include <iomanip>
#include <iostream>
#include <vector>
#include <Compression/CompressedReadBuffer.h>
#include <Core/Types.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <Interpreters/AggregationCommon.h>
#include <Common/HashTable/HashMap.h>
#include <Common/HashTable/HashTableKeyHolder.h>
#include <Common/HashTable/StringHashMap.h>
#include <Common/Stopwatch.h>
#include <common/StringRef.h>
/**
#include <fstream>
#include <random>
using namespace std;
int main()
{
std::string s;
std::random_device dev;
std::mt19937 rng(dev());
std::uniform_int_distribution<std::mt19937::result_type> dist(0, 25);
std::binomial_distribution<std::mt19937::result_type> binomial1(100, 0.01);
std::binomial_distribution<std::mt19937::result_type> binomial2(100, 0.02);
std::binomial_distribution<std::mt19937::result_type> binomial4(100, 0.04);
std::binomial_distribution<std::mt19937::result_type> binomial8(100, 0.08);
std::binomial_distribution<std::mt19937::result_type> binomial16(100, 0.16);
std::binomial_distribution<std::mt19937::result_type> binomial24(100, 0.24);
std::binomial_distribution<std::mt19937::result_type> binomial48(100, 0.48);
// 11GB
std::ofstream f("/tmp/terms.csv");
size_t l1, l2, l4, l8, l16, l24, l48;
for (auto n = 0ul; n < 1e8; ++n)
{
l1 = binomial1(rng) + 1;
l2 = binomial2(rng) + l1 + 1;
l4 = binomial4(rng) + l2 + 1;
l8 = binomial8(rng) + l4 + 1;
l16 = binomial16(rng) + l8 + 1;
l24 = binomial24(rng) + l16 + 1;
l48 = binomial48(rng) + l24 + 1;
s.resize(l48);
for (auto i = 0ul; i < l48 - 1; ++i)
s[i] = 'a' + dist(rng);
s[l1 - 1] = ',';
s[l2 - 1] = ',';
s[l4 - 1] = ',';
s[l8 - 1] = ',';
s[l16 - 1] = ',';
s[l24 - 1] = ',';
s[l48 - 1] = '\n';
f << s;
}
f.close();
return 0;
}
create table terms (term1 String, term2 String, term4 String, term8 String, term16 String, term24 String, term48 String) engine TinyLog;
insert into terms select * from file('/tmp/terms.csv', CSV, 'a String, b String, c String, d String, e String, f String, g String');
NOTE: for reliable test results, try isolating cpu cores and do python -m perf tune. Also bind numa nodes if any.
# isolate cpu 18
dir=/home/amos/git/chorigin/data/data/default/terms
for file in term1 term2 term4 term8 term16 term24 term48; do
for size in 30000000 50000000 80000000 100000000; do
BEST_METHOD=0
BEST_RESULT=0
for method in {1..2}; do
echo -ne $file $size $method ''
numactl --membind=0 taskset -c 18 ./string_hash_map $size $method <"$dir"/"$file".bin 2>&1 | perl -nE 'say /([0-9\.]+) elem/g if /HashMap/' | tee /tmp/string_hash_map_res
CUR_RESULT=$(cat /tmp/string_hash_map_res | tr -d '.')
if [[ $CUR_RESULT -gt $BEST_RESULT ]]; then
BEST_METHOD=$method
BEST_RESULT=$CUR_RESULT
fi
done
echo Best: $BEST_METHOD - $BEST_RESULT
done
done
---------------------------
term1 30000000 1 68785770.85 term2 30000000 1 42531788.83 term4 30000000 1 14759901.41 term8 30000000 1 8072903.47
term1 30000000 2 40812128.16 term2 30000000 2 21352402.10 term4 30000000 2 9008907.80 term8 30000000 2 5822641.82
Best: 1 - 6878577085 Best: 1 - 4253178883 Best: 1 - 1475990141 Best: 1 - 807290347
term1 50000000 1 68027542.41 term2 50000000 1 40493742.80 term4 50000000 1 16827650.85 term8 50000000 1 7405230.14
term1 50000000 2 37589806.02 term2 50000000 2 19362975.09 term4 50000000 2 8278094.11 term8 50000000 2 5106810.80
Best: 1 - 6802754241 Best: 1 - 4049374280 Best: 1 - 1682765085 Best: 1 - 740523014
term1 80000000 1 68651875.88 term2 80000000 1 38253695.50 term4 80000000 1 15847177.93 term8 80000000 1 7536319.25
term1 80000000 2 38092189.20 term2 80000000 2 20287003.01 term4 80000000 2 9322770.34 term8 80000000 2 4355572.15
Best: 1 - 6865187588 Best: 1 - 3825369550 Best: 1 - 1584717793 Best: 1 - 753631925
term1 100000000 1 68641941.59 term2 100000000 1 39120834.79 term4 100000000 1 16773904.90 term8 100000000 1 7147146.55
term1 100000000 2 38358006.72 term2 100000000 2 20629363.17 term4 100000000 2 9665201.92 term8 100000000 2 4728255.07
Best: 1 - 6864194159 Best: 1 - 3912083479 Best: 1 - 1677390490 Best: 1 - 714714655
term16 30000000 1 6823029.35 term24 30000000 1 5706271.14 term48 30000000 1 4695716.47
term16 30000000 2 5672283.33 term24 30000000 2 5498855.56 term48 30000000 2 4860537.26
Best: 1 - 682302935 Best: 1 - 570627114 Best: 2 - 486053726
term16 50000000 1 6214581.25 term24 50000000 1 5249785.66 term48 50000000 1 4282606.12
term16 50000000 2 4990361.44 term24 50000000 2 4855552.24 term48 50000000 2 4348923.29
Best: 1 - 621458125 Best: 1 - 524978566 Best: 2 - 434892329
term16 80000000 1 5382855.70 term24 80000000 1 4580133.04 term48 80000000 1 3779436.15
term16 80000000 2 4282192.79 term24 80000000 2 4178791.09 term48 80000000 2 3788409.72
Best: 1 - 538285570 Best: 1 - 458013304 Best: 2 - 378840972
term16 100000000 1 5930103.42 term24 100000000 1 5030621.52 term48 100000000 1 4084666.73
term16 100000000 2 4621719.60 term24 100000000 2 4499866.83 term48 100000000 2 4067029.31
Best: 1 - 593010342 Best: 1 - 503062152 Best: 1 - 408466673
*/
using Value = uint64_t;
template <typename Map>
void NO_INLINE bench(const std::vector<StringRef> & data, DB::Arena &, const char * name)
{
// warm up
/*
{
Map map;
typename Map::LookupResult it;
bool inserted;
for (size_t i = 0, size = data.size(); i < size; ++i)
{
auto key_holder = DB::ArenaKeyHolder{data[i], pool};
map.emplace(key_holder, it, inserted);
if (inserted)
it->getSecond() = 0;
++it->getSecond();
}
}
*/
std::cerr << "method " << name << std::endl;
for (auto t = 0ul; t < 7; ++t)
{
DB::Arena pool(128 * 1024 * 1024);
Stopwatch watch;
Map map;
typename Map::LookupResult it;
bool inserted;
for (size_t i = 0, size = data.size(); i < size; ++i)
{
map.emplace(DB::ArenaKeyHolder{data[i], pool}, it, inserted);
if (inserted)
*lookupResultGetMapped(it) = 0;
++*lookupResultGetMapped(it);
}
watch.stop();
std::cerr << "arena-memory " << pool.size() + map.getBufferSizeInBytes() << std::endl;
std::cerr << "single-run " << std::setprecision(3)
<< watch.elapsedSeconds() << std::endl;
}
}
/*
template <typename Map>
runFromFile()
{
DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
DB::CompressedReadBuffer in2(in1);
Map map;
DB::Arena pool(128 * 1024 * 1024);
for (size_t i = 0; i < n && !in2.eof(); ++i)
{
auto key = DB::readStringBinaryInto(pool, in2);
bool inserted;
Map::LookupResult mapped;
map.emplaceKeyHolder(DB::SerializedKeyHolder(key, pool), mapped, inserted);
}
}
template <typename Map>
benchFromFile()
{
double best_time = -1.;
for (auto t = 0ul; t < 50; ++t)
{
Stopwatch watch;
runFromFile();
watch.stop();
if (best_time < 0 || best_time > watch.elapsedSeconds())
{
best_time = watch.elapsedSeconds();
}
}
std::cerr << std::fixed << std::setprecision(2) << "HashMap (" << name << "). Elapsed: " << best_time << " (" << data.size() / best_time
<< " elem/sec.)" << std::endl;
}
*/
int main(int argc, char ** argv)
{
if (argc < 3)
{
std::cerr << "Usage: program n m\n";
return 1;
}
size_t n = atoi(argv[1]);
size_t m = atoi(argv[2]);
DB::Arena pool(128 * 1024 * 1024);
std::vector<StringRef> data(n);
std::cerr << "sizeof(Key) = " << sizeof(StringRef) << ", sizeof(Value) = " << sizeof(Value) << std::endl;
{
Stopwatch watch;
DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
DB::CompressedReadBuffer in2(in1);
std::string tmp;
for (size_t i = 0; i < n && !in2.eof(); ++i)
{
DB::readStringBinary(tmp, in2);
data[i] = StringRef(pool.insert(tmp.data(), tmp.size()), tmp.size());
}
watch.stop();
std::cerr << std::fixed << std::setprecision(2) << "Vector. Size: " << n << ", elapsed: " << watch.elapsedSeconds() << " ("
<< n / watch.elapsedSeconds() << " elem/sec.)" << std::endl;
}
if (!m || m == 1)
bench<StringHashMap<Value>>(data, pool, "StringHashMap");
if (!m || m == 2)
bench<HashMapWithSavedHash<StringRef, Value>>(data, pool, "HashMapWithSavedHash");
if (!m || m == 3)
bench<HashMap<StringRef, Value>>(data, pool, "HashMap");
return 0;
}

View File

@ -20,9 +20,6 @@ public:
ASTPtr table_function;
ASTPtr settings_ast;
// Set to true if the data should only be inserted into attached views
bool no_destination = false;
/// Data to insert
const char * data = nullptr;
const char * end = nullptr;

View File

@ -102,6 +102,9 @@ public:
/// Returns true if the storage supports settings.
virtual bool supportsSettings() const { return false; }
/// Returns true if the blocks shouldn't be pushed to associated views on insert.
virtual bool noPushingToViews() const { return false; }
/// Optional size information of each physical column.
/// Currently it's only used by the MergeTree family for query optimizations.
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;

View File

@ -0,0 +1,14 @@
#pragma once
#include <memory>
namespace DB
{
class ReadBufferFromKafkaConsumer;
class WriteBufferToKafkaProducer;
using ConsumerBufferPtr = std::shared_ptr<ReadBufferFromKafkaConsumer>;
using ProducerBufferPtr = std::shared_ptr<WriteBufferToKafkaProducer>;
}

View File

@ -1,6 +1,7 @@
#include "KafkaBlockOutputStream.h"
#include <Formats/FormatFactory.h>
#include <Storages/Kafka/WriteBufferToKafkaProducer.h>
namespace DB
{

View File

@ -2,6 +2,8 @@
#include <common/logger_useful.h>
#include <cppkafka/cppkafka.h>
namespace DB
{

View File

@ -61,6 +61,4 @@ private:
bool nextImpl() override;
};
using ConsumerBufferPtr = std::shared_ptr<ReadBufferFromKafkaConsumer>;
}

View File

@ -17,6 +17,7 @@
#include <Storages/Kafka/KafkaSettings.h>
#include <Storages/Kafka/KafkaBlockInputStream.h>
#include <Storages/Kafka/KafkaBlockOutputStream.h>
#include <Storages/Kafka/WriteBufferToKafkaProducer.h>
#include <Storages/StorageFactory.h>
#include <Storages/StorageMaterializedView.h>
#include <boost/algorithm/string/replace.hpp>
@ -366,7 +367,6 @@ bool StorageKafka::streamToViews()
auto insert = std::make_shared<ASTInsertQuery>();
insert->database = database_name;
insert->table = table_name;
insert->no_destination = true; // Only insert into dependent views and expect that input blocks contain virtual columns
const Settings & settings = global_context.getSettingsRef();
size_t block_size = max_block_size;
@ -374,7 +374,8 @@ bool StorageKafka::streamToViews()
block_size = settings.max_block_size;
// Create a stream for each consumer and join them in a union stream
InterpreterInsertQuery interpreter(insert, global_context, false, true);
// Only insert into dependent views and expect that input blocks contain virtual columns
InterpreterInsertQuery interpreter(insert, global_context, false, true, true);
auto block_io = interpreter.execute();
// Create a stream for each consumer and join them in a union stream

View File

@ -2,8 +2,7 @@
#include <Core/BackgroundSchedulePool.h>
#include <Storages/IStorage.h>
#include <Storages/Kafka/ReadBufferFromKafkaConsumer.h>
#include <Storages/Kafka/WriteBufferToKafkaProducer.h>
#include <Storages/Kafka/Buffer_fwd.h>
#include <Poco/Semaphore.h>
#include <ext/shared_ptr_helper.h>
@ -11,6 +10,12 @@
#include <mutex>
#include <atomic>
namespace cppkafka
{
class Configuration;
}
namespace DB
{
@ -25,7 +30,9 @@ public:
std::string getName() const override { return "Kafka"; }
std::string getTableName() const override { return table_name; }
std::string getDatabaseName() const override { return database_name; }
bool supportsSettings() const override { return true; }
bool noPushingToViews() const override { return true; }
void startup() override;
void shutdown() override;

View File

@ -9,9 +9,6 @@
namespace DB
{
class WriteBufferToKafkaProducer;
using ProducerBufferPtr = std::shared_ptr<WriteBufferToKafkaProducer>;
using ProducerPtr = std::shared_ptr<cppkafka::Producer>;
class WriteBufferToKafkaProducer : public WriteBuffer

View File

@ -3043,7 +3043,7 @@ DiskSpace::ReservationPtr MergeTreeData::reserveSpace(UInt64 expected_size)
if (reservation)
return reservation;
throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(expected_size) + ", not enought space.",
throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(expected_size) + ", not enough space.",
ErrorCodes::NOT_ENOUGH_SPACE);
}

View File

@ -323,11 +323,13 @@ BlockInputStreams StorageDistributed::read(
Block header =
InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock();
const Scalars & scalars = context.hasQueryContext() ? context.getQueryContext().getScalars() : Scalars{};
ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr
? ClusterProxy::SelectStreamFactory(
header, processed_stage, remote_table_function_ptr, context.getExternalTables())
header, processed_stage, remote_table_function_ptr, scalars, context.getExternalTables())
: ClusterProxy::SelectStreamFactory(
header, processed_stage, QualifiedTableName{remote_database, remote_table}, context.getExternalTables());
header, processed_stage, QualifiedTableName{remote_database, remote_table}, scalars, context.getExternalTables());
if (settings.optimize_skip_unused_shards)
{

View File

@ -5,6 +5,7 @@ const char * auto_contributors[] {
"Akazz",
"Alberto",
"Aleksandra (Ася)",
"Aleksey Akulovich",
"Alex Bocharov",
"Alex Krash",
"Alex Ryndin",
@ -90,7 +91,9 @@ const char * auto_contributors[] {
"Daniel Bershatsky",
"Daniel Dao",
"Danila Kutenin",
"Dario",
"DarkWanderer",
"Darío",
"Denis Burlaka",
"Denis Zhuravlev",
"Derek Perkins",
@ -103,6 +106,7 @@ const char * auto_contributors[] {
"Dmitry S..ky / skype: dvska-at-skype",
"Doge",
"Eldar Zaitov",
"Elena Baskakova",
"Elghazal Ahmed",
"Emmanuel Donin de Rosière",
"Eric",
@ -114,6 +118,7 @@ const char * auto_contributors[] {
"Evgeny Konkov",
"Fabian Stäber",
"Fadi Hadzh",
"FawnD2",
"FeehanG",
"Flowyi",
"Francisco Barón",
@ -124,6 +129,7 @@ const char * auto_contributors[] {
"George3d6",
"Gleb Kanterov",
"Gleb Novikov",
"Gleb-Tretyakov",
"Guillaume Tassery",
"Hamoon",
"Hasnat",
@ -140,6 +146,7 @@ const char * auto_contributors[] {
"Ilya Breev",
"Ilya Khomutov",
"Ilya Korolev",
"Ilya Kovalenko",
"Ilya Shipitsin",
"Ilya Skrypitsa",
"Ivan",
@ -208,6 +215,7 @@ const char * auto_contributors[] {
"Mikhail",
"Mikhail Fandyushin",
"Mikhail Filimonov",
"Mikhail Korotov",
"Mikhail Salosin",
"Mikhail Surin",
"Mikhail f. Shiryaev",
@ -246,6 +254,7 @@ const char * auto_contributors[] {
"Pavlo Bashynskiy",
"Pawel Rog",
"Persiyanov Dmitriy Andreevich",
"Philippe Ombredanne",
"Quid37",
"Rafael David Tinoco",
"Ramazan Polat",
@ -296,6 +305,7 @@ const char * auto_contributors[] {
"Vasily Nemkov",
"Vasily Okunev",
"Vasily Vasilkov",
"Vasilyev Nikita",
"Veloman Yunkan",
"Veniamin Gvozdikov",
"Victor Tarnavsky",
@ -308,6 +318,7 @@ const char * auto_contributors[] {
"Vlad Arkhipov",
"Vladimir Chebotarev",
"Vladimir Kolobaev",
"Vladimir Kopysov",
"Vladimir Kozbin",
"Vladimir Smirnov",
"Vladislav Rassokhin",
@ -336,6 +347,7 @@ const char * auto_contributors[] {
"akuzm",
"alesapin",
"alex-zaitsev",
"alex.lvxin",
"alexander kozhikhov",
"alexey-milovidov",
"andrewsg",
@ -355,6 +367,7 @@ const char * auto_contributors[] {
"chenxing-xc",
"chenxing.xc",
"chertus",
"comunodi",
"coraxster",
"daoready",
"dasmfm",
@ -449,6 +462,8 @@ const char * auto_contributors[] {
"urgordeadbeef",
"velom",
"vicdashkov",
"vinity",
"vzakaznikov",
"zamulla",
"zhang2014",
"Šimon Podlipský",

View File

@ -18,6 +18,7 @@ create_table_sql_template = """
`name` varchar(50) NOT NULL,
`age` int NOT NULL default 0,
`money` int NOT NULL default 0,
`column_x` int default NULL,
PRIMARY KEY (`id`)) ENGINE=InnoDB;
"""
def get_mysql_conn():
@ -86,12 +87,21 @@ def test_mysql_simple_select_works(started_cluster):
conn = get_mysql_conn()
create_mysql_table(conn, table_name)
# Check that NULL-values are handled correctly by the ODBC-bridge
with conn.cursor() as cursor:
cursor.execute("INSERT INTO clickhouse.{} VALUES(50, 'null-guy', 127, 255, NULL), (100, 'non-null-guy', 127, 255, 511);".format(table_name))
conn.commit()
assert node1.query("SELECT column_x FROM odbc('DSN={}', '{}')".format(mysql_setup["DSN"], table_name)) == '\\N\n511\n'
node1.query('''
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse');
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nullable(UInt32)) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse');
'''.format(table_name, table_name))
node1.query("INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(100) ".format(table_name))
node1.query("INSERT INTO {}(id, name, money, column_x) select number, concat('name_', toString(number)), 3, NULL from numbers(49) ".format(table_name))
node1.query("INSERT INTO {}(id, name, money, column_x) select number, concat('name_', toString(number)), 3, 42 from numbers(51, 49) ".format(table_name))
assert node1.query("SELECT COUNT () FROM {} WHERE column_x IS NOT NULL".format(table_name)) == '50\n'
assert node1.query("SELECT COUNT () FROM {} WHERE column_x IS NULL".format(table_name)) == '50\n'
assert node1.query("SELECT count(*) FROM odbc('DSN={}', '{}')".format(mysql_setup["DSN"], table_name)) == '100\n'
# previously this test fails with segfault

View File

@ -560,6 +560,8 @@ def test_kafka_insert(kafka_cluster):
@pytest.mark.timeout(180)
def test_kafka_produce_consume(kafka_cluster):
instance.query('''
DROP TABLE IF EXISTS test.view;
DROP TABLE IF EXISTS test.consumer;
CREATE TABLE test.kafka (key UInt64, value UInt64)
ENGINE = Kafka
SETTINGS kafka_broker_list = 'kafka1:19092',
@ -567,6 +569,11 @@ def test_kafka_produce_consume(kafka_cluster):
kafka_group_name = 'insert2',
kafka_format = 'TSV',
kafka_row_delimiter = '\\n';
CREATE TABLE test.view (key UInt64, value UInt64)
ENGINE = MergeTree
ORDER BY key;
CREATE MATERIALIZED VIEW test.consumer TO test.view AS
SELECT * FROM test.kafka;
''')
messages_num = 10000
@ -594,16 +601,6 @@ def test_kafka_produce_consume(kafka_cluster):
time.sleep(random.uniform(0, 1))
thread.start()
instance.query('''
DROP TABLE IF EXISTS test.view;
DROP TABLE IF EXISTS test.consumer;
CREATE TABLE test.view (key UInt64, value UInt64)
ENGINE = MergeTree
ORDER BY key;
CREATE MATERIALIZED VIEW test.consumer TO test.view AS
SELECT * FROM test.kafka;
''')
while True:
result = instance.query('SELECT count() FROM test.view')
time.sleep(1)

View File

@ -0,0 +1,35 @@
<test>
<type>loop</type>
<stop_conditions>
<all_of>
<total_time_ms>30000</total_time_ms>
</all_of>
<any_of>
<min_time_not_changing_for_ms>5000</min_time_not_changing_for_ms>
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<create_query>CREATE TABLE cdp_tags (tag_id String, mid_seqs AggregateFunction(groupBitmap, UInt32)) engine=MergeTree() ORDER BY (tag_id) SETTINGS index_granularity=1</create_query>
<create_query>CREATE TABLE cdp_orders(order_id UInt64, order_complete_time DateTime, order_total_sales Float32, mid_seq UInt32) engine=MergeTree() PARTITION BY toYYYYMMDD(order_complete_time) ORDER BY (order_complete_time, order_id)</create_query>
<fill_query>INSERT INTO cdp_tags(tag_id, mid_seqs) SELECT 'tag1', groupBitmapState(toUInt32(number)) FROM numbers(10000000) WHERE number%9=0</fill_query>
<fill_query>INSERT INTO cdp_tags(tag_id, mid_seqs) SELECT 'tag2', groupBitmapState(toUInt32(number)) FROM numbers(10000000) WHERE number%8=0</fill_query>
<fill_query>INSERT INTO cdp_tags(tag_id, mid_seqs) SELECT 'tag3', groupBitmapState(toUInt32(number)) FROM numbers(10000000) WHERE number%7=0</fill_query>
<fill_query>INSERT INTO cdp_tags(tag_id, mid_seqs) SELECT 'tag4', groupBitmapState(toUInt32(number)) FROM numbers(10000000) WHERE number%6=0</fill_query>
<fill_query>INSERT INTO cdp_tags(tag_id, mid_seqs) SELECT 'tag5', groupBitmapState(toUInt32(number)) FROM numbers(10000000) WHERE number%5=0</fill_query>
<fill_query>INSERT INTO cdp_tags(tag_id, mid_seqs) SELECT 'tag6', groupBitmapState(toUInt32(number)) FROM numbers(10000000) WHERE number%4=0</fill_query>
<fill_query>INSERT INTO cdp_tags(tag_id, mid_seqs) SELECT 'tag7', groupBitmapState(toUInt32(number)) FROM numbers(10000000) WHERE number%3=0</fill_query>
<fill_query>INSERT INTO cdp_tags(tag_id, mid_seqs) SELECT 'tag8', groupBitmapState(toUInt32(number)) FROM numbers(10000000) WHERE number%2=0</fill_query>
<fill_query>INSERT INTO cdp_orders(order_id, order_complete_time, order_total_sales, mid_seq) SELECT number, addSeconds(toDateTime('2000-01-01 00:00:00'), number), number%1024, toUInt32(number) FROM numbers(10000000)</fill_query>
<query>WITH (SELECT mid_seqs FROM cdp_tags WHERE tag_id='tag1') AS bm1, (SELECT mid_seqs FROM cdp_tags WHERE tag_id='tag2') AS bm2, (SELECT mid_seqs FROM cdp_tags WHERE tag_id='tag3') AS bm3, (SELECT mid_seqs FROM cdp_tags WHERE tag_id='tag4') AS bm4, (SELECT mid_seqs FROM cdp_tags WHERE tag_id='tag5') AS bm5, (SELECT mid_seqs FROM cdp_tags WHERE tag_id='tag6') AS bm6, (SELECT mid_seqs FROM cdp_tags WHERE tag_id='tag7') AS bm7, (SELECT mid_seqs FROM cdp_tags WHERE tag_id='tag8') AS bm8, toDateTime('2000-01-01 00:00:00') AS ts_begin, addSeconds(toDateTime('2000-01-01 00:00:00'), 1e8) AS ts_end SELECT multiIf(bitmapContains(bm1, mid_seq), 1, bitmapContains(bm2, mid_seq), 2, bitmapContains(bm3, mid_seq), 3, bitmapContains(bm4, mid_seq), 4, bitmapContains(bm5, mid_seq), 5, bitmapContains(bm6, mid_seq), 6, bitmapContains(bm7, mid_seq), 7, bitmapContains(bm8, mid_seq), 8, 0) AS tag, count() AS gc, sum(order_total_sales) AS total FROM cdp_orders PREWHERE order_complete_time >= ts_begin AND order_complete_time &lt; ts_end GROUP BY tag ORDER BY tag</query>
<drop_query>DROP TABLE IF EXISTS cdp_tags</drop_query>
<drop_query>DROP TABLE IF EXISTS cdp_orders</drop_query>
</test>

File diff suppressed because it is too large Load Diff

View File

@ -1 +1,2 @@
SELECT number, exp10(number) FROM system.numbers LIMIT 310;
SELECT number, exp10(number - 500) FROM system.numbers LIMIT 1000;
SELECT exp10(nan);

View File

@ -0,0 +1,58 @@
0 0 0
[0.0000,0.0000] [0.0000000,0.0000000] [0.00000000,0.00000000]
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 Decimal(9, 4) Decimal(18, 7) Decimal(38, 8)
0.0000 0.0000000 0.00000000 Decimal(9, 4) Decimal(18, 7) Decimal(38, 8)
0.0000 0.0000000 0.00000000 Decimal(9, 4) Decimal(18, 7) Decimal(38, 8)
(0,0,0) (0,0,0) (0,0,0) (0,0,0) (0,0,0)
0 0 0
0 0 0
0.0000 0.0000 0.0000000 0.0000000 0.00000000 0.00000000
0.0000 0.0000 0.0000000 0.0000000 0.00000000 0.00000000
0.0000 0.0000 0.0000000 0.0000000 0.00000000 0.00000000
0.0000 0.0000 0.0000000 0.0000000 0.00000000 0.00000000
0.0000 0.0000000 0.00000000 Decimal(6, 4) Decimal(16, 7) Decimal(20, 8)
0.0000 0.0000000 0.00000000 Decimal(6, 4) Decimal(16, 7) Decimal(20, 8)
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
[0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000]
[0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000]
[0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000]
0.0000 0.0000000 0.00000000 Decimal(20, 8)
0.0000 0.0000000 0.00000000 Decimal(20, 8)
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
[0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000]
[0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000]
[0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000]
0.0000 0.0000000 0.00000000 Decimal(20, 8)
0.0000 0.0000000 0.00000000 Decimal(20, 8)
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
[0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000]
[0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000]
[0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000]
inf inf inf Float64 Float64 Float64
nan nan nan
nan nan nan Float64 Float64 Float64
nan nan nan
inf inf inf Float64 Float64 Float64
nan nan nan
nan nan nan Float64 Float64 Float64
nan nan nan

View File

@ -0,0 +1,90 @@
DROP TABLE IF EXISTS decimal;
CREATE TABLE decimal
(
a Decimal(6, 4),
b Decimal(16, 7),
c Decimal(20, 8)
) ENGINE = Memory;
SELECT count(a), count(b), count(c) FROM decimal;
SELECT [min(a), max(a)], [min(b), max(b)], [min(c), max(c)] FROM decimal;
SELECT sum(a), sum(b), sum(c), sumWithOverflow(a), sumWithOverflow(b), sumWithOverflow(c) FROM decimal;
SELECT sum(a), sum(b), sum(c), sumWithOverflow(a), sumWithOverflow(b), sumWithOverflow(c) FROM decimal WHERE a > 0;
SELECT sum(a), sum(b), sum(c), sumWithOverflow(a), sumWithOverflow(b), sumWithOverflow(c) FROM decimal WHERE a < 0;
SELECT sum(a+1), sum(b+1), sum(c+1), sumWithOverflow(a+1), sumWithOverflow(b+1), sumWithOverflow(c+1) FROM decimal;
SELECT sum(a-1), sum(b-1), sum(c-1), sumWithOverflow(a-1), sumWithOverflow(b-1), sumWithOverflow(c-1) FROM decimal;
SELECT avg(a) as aa, avg(b) as ab, avg(c) as ac, toTypeName(aa), toTypeName(ab),toTypeName(ac) FROM decimal;
SELECT avg(a) as aa, avg(b) as ab, avg(c) as ac, toTypeName(aa), toTypeName(ab),toTypeName(ac) FROM decimal WHERE a > 0;
SELECT avg(a) as aa, avg(b) as ab, avg(c) as ac, toTypeName(aa), toTypeName(ab),toTypeName(ac) FROM decimal WHERE a < 0;
SELECT (uniq(a), uniq(b), uniq(c)),
(uniqCombined(a), uniqCombined(b), uniqCombined(c)),
(uniqCombined(17)(a), uniqCombined(17)(b), uniqCombined(17)(c)),
(uniqExact(a), uniqExact(b), uniqExact(c)),
(uniqHLL12(a), uniqHLL12(b), uniqHLL12(c))
FROM (SELECT * FROM decimal ORDER BY a);
SELECT uniqUpTo(10)(a), uniqUpTo(10)(b), uniqUpTo(10)(c) FROM decimal WHERE a >= 0 AND a < 5;
SELECT uniqUpTo(10)(a), uniqUpTo(10)(b), uniqUpTo(10)(c) FROM decimal WHERE a >= 0 AND a < 10;
SELECT argMin(a, b), argMin(a, c), argMin(b, a), argMin(b, c), argMin(c, a), argMin(c, b) FROM decimal;
SELECT argMin(a, b), argMin(a, c), argMin(b, a), argMin(b, c), argMin(c, a), argMin(c, b) FROM decimal WHERE a > 0;
SELECT argMax(a, b), argMax(a, c), argMax(b, a), argMax(b, c), argMax(c, a), argMax(c, b) FROM decimal;
SELECT argMax(a, b), argMax(a, c), argMax(b, a), argMax(b, c), argMax(c, a), argMax(c, b) FROM decimal WHERE a < 0;
SELECT median(a) as ma, median(b) as mb, median(c) as mc, toTypeName(ma),toTypeName(mb),toTypeName(mc) FROM decimal;
SELECT quantile(a) as qa, quantile(b) as qb, quantile(c) as qc, toTypeName(qa),toTypeName(qb),toTypeName(qc) FROM decimal WHERE a < 0;
SELECT quantile(0.0)(a), quantile(0.0)(b), quantile(0.0)(c) FROM decimal WHERE a >= 0;
SELECT quantile(0.2)(a), quantile(0.2)(b), quantile(0.2)(c) FROM decimal WHERE a >= 0;
SELECT quantile(0.4)(a), quantile(0.4)(b), quantile(0.4)(c) FROM decimal WHERE a >= 0;
SELECT quantile(0.6)(a), quantile(0.6)(b), quantile(0.6)(c) FROM decimal WHERE a >= 0;
SELECT quantile(0.8)(a), quantile(0.8)(b), quantile(0.8)(c) FROM decimal WHERE a >= 0;
SELECT quantile(1.0)(a), quantile(1.0)(b), quantile(1.0)(c) FROM decimal WHERE a >= 0;
SELECT quantiles(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(a) FROM decimal;
SELECT quantiles(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(b) FROM decimal;
SELECT quantiles(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(c) FROM decimal;
SELECT medianExact(a), medianExact(b), medianExact(c) as x, toTypeName(x) FROM decimal;
SELECT quantileExact(a), quantileExact(b), quantileExact(c) as x, toTypeName(x) FROM decimal WHERE a < 0;
SELECT quantileExact(0.0)(a), quantileExact(0.0)(b), quantileExact(0.0)(c) FROM decimal WHERE a >= 0;
SELECT quantileExact(0.2)(a), quantileExact(0.2)(b), quantileExact(0.2)(c) FROM decimal WHERE a >= 0;
SELECT quantileExact(0.4)(a), quantileExact(0.4)(b), quantileExact(0.4)(c) FROM decimal WHERE a >= 0;
SELECT quantileExact(0.6)(a), quantileExact(0.6)(b), quantileExact(0.6)(c) FROM decimal WHERE a >= 0;
SELECT quantileExact(0.8)(a), quantileExact(0.8)(b), quantileExact(0.8)(c) FROM decimal WHERE a >= 0;
SELECT quantileExact(1.0)(a), quantileExact(1.0)(b), quantileExact(1.0)(c) FROM decimal WHERE a >= 0;
SELECT quantilesExact(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(a) FROM decimal;
SELECT quantilesExact(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(b) FROM decimal;
SELECT quantilesExact(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(c) FROM decimal;
SELECT medianExactWeighted(a, 1), medianExactWeighted(b, 2), medianExactWeighted(c, 3) as x, toTypeName(x) FROM decimal;
SELECT quantileExactWeighted(a, 1), quantileExactWeighted(b, 2), quantileExactWeighted(c, 3) as x, toTypeName(x) FROM decimal WHERE a < 0;
SELECT quantileExactWeighted(0.0)(a, 1), quantileExactWeighted(0.0)(b, 2), quantileExactWeighted(0.0)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantileExactWeighted(0.2)(a, 1), quantileExactWeighted(0.2)(b, 2), quantileExactWeighted(0.2)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantileExactWeighted(0.4)(a, 1), quantileExactWeighted(0.4)(b, 2), quantileExactWeighted(0.4)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantileExactWeighted(0.6)(a, 1), quantileExactWeighted(0.6)(b, 2), quantileExactWeighted(0.6)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantileExactWeighted(0.8)(a, 1), quantileExactWeighted(0.8)(b, 2), quantileExactWeighted(0.8)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantileExactWeighted(1.0)(a, 1), quantileExactWeighted(1.0)(b, 2), quantileExactWeighted(1.0)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantilesExactWeighted(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(a, 1) FROM decimal;
SELECT quantilesExactWeighted(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(b, 2) FROM decimal;
SELECT quantilesExactWeighted(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(c, 3) FROM decimal;
SELECT varPop(a) AS va, varPop(b) AS vb, varPop(c) AS vc, toTypeName(va), toTypeName(vb), toTypeName(vc) FROM decimal;
SELECT varPop(toFloat64(a)), varPop(toFloat64(b)), varPop(toFloat64(c)) FROM decimal;
SELECT varSamp(a) AS va, varSamp(b) AS vb, varSamp(c) AS vc, toTypeName(va), toTypeName(vb), toTypeName(vc) FROM decimal;
SELECT varSamp(toFloat64(a)), varSamp(toFloat64(b)), varSamp(toFloat64(c)) FROM decimal;
SELECT stddevPop(a) AS da, stddevPop(b) AS db, stddevPop(c) AS dc, toTypeName(da), toTypeName(db), toTypeName(dc) FROM decimal;
SELECT stddevPop(toFloat64(a)), stddevPop(toFloat64(b)), stddevPop(toFloat64(c)) FROM decimal;
SELECT stddevSamp(a) AS da, stddevSamp(b) AS db, stddevSamp(c) AS dc, toTypeName(da), toTypeName(db), toTypeName(dc) FROM decimal;
SELECT stddevSamp(toFloat64(a)), stddevSamp(toFloat64(b)), stddevSamp(toFloat64(c)) FROM decimal;
SELECT covarPop(a, a), covarPop(b, b), covarPop(c, c) FROM decimal; -- { serverError 43 }
SELECT covarSamp(a, a), covarSamp(b, b), covarSamp(c, c) FROM decimal; -- { serverError 43 }
SELECT corr(a, a), corr(b, b), corr(c, c) FROM decimal; -- { serverError 43 }
SELECT 1 LIMIT 0;
DROP TABLE decimal;

View File

@ -7,6 +7,10 @@
1
1
0
--isValidJSON--
1
0
0
--JSONKey--
a
b

View File

@ -11,6 +11,11 @@ SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'a');
SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'b');
SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'c');
SELECT '--isValidJSON--';
SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}');
SELECT isValidJSON('not a json');
SELECT isValidJSON('"HX-=');
SELECT '--JSONKey--';
SELECT JSONKey('{"a": "hello", "b": [-100, 200.0, 300]}', 1);
SELECT JSONKey('{"a": "hello", "b": [-100, 200.0, 300]}', 2);

View File

@ -93,7 +93,7 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dic
LIFETIME(MIN 1 MAX 10)
LAYOUT(FLAT()); -- {serverError 1}
SHOW CREATE DICTIONARY memory_db.dict2; -- {serverError 485}
SHOW CREATE DICTIONARY memory_db.dict2; -- {serverError 487}
SHOW DICTIONARIES FROM memory_db LIKE 'dict2';

View File

@ -0,0 +1,35 @@
DROP TABLE IF EXISTS mv_extra_columns_dst;
DROP TABLE IF EXISTS mv_extra_columns_src;
DROP TABLE IF EXISTS mv_extra_columns_view;
CREATE TABLE mv_extra_columns_dst (
v UInt64
) ENGINE = MergeTree()
PARTITION BY tuple()
ORDER BY v;
CREATE TABLE mv_extra_columns_src (
v1 UInt64,
v2 UInt64
) ENGINE = Null;
-- Extra columns are ignored when pushing to destination table.
-- This test exists to prevent unintended changes to existing behaviour.
--
-- Although this behaviour might not be ideal it can be exploited for 0-downtime changes to materialized views.
-- Step 1: Add new column to source table. Step 2: Create new view reading source column.
-- Step 3: Swap views using `RENAME TABLE`. Step 4: Add new column to destination table as well.
CREATE MATERIALIZED VIEW mv_extra_columns_view TO mv_extra_columns_dst
AS SELECT
v1 as v,
v2 as v2
FROM mv_extra_columns_src;
INSERT INTO mv_extra_columns_src VALUES (0, 0), (1, 1), (2, 2);
SELECT * FROM mv_extra_columns_dst ORDER by v;
SELECT * FROM mv_extra_columns_view; -- { serverError 16 }
DROP TABLE mv_extra_columns_view;
DROP TABLE mv_extra_columns_src;
DROP TABLE mv_extra_columns_dst;

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (19.16.1.1) unstable; urgency=low
clickhouse (19.17.1.1) unstable; urgency=low
* Modified source code
-- clickhouse-release <clickhouse-release@yandex-team.ru> Tue, 24 Sep 2019 14:47:28 +0300
-- clickhouse-release <clickhouse-release@yandex-team.ru> Mon, 21 Oct 2019 15:47:56 +0300

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=19.16.1.*
ARG version=19.17.1.*
RUN apt-get update \
&& apt-get install --yes --no-install-recommends \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=19.16.1.*
ARG version=19.17.1.*
ARG gosu_ver=1.10
RUN apt-get update \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=19.16.1.*
ARG version=19.17.1.*
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -321,12 +321,11 @@ ORDER BY Year;
Q8. The most popular destinations by the number of directly connected cities for various year ranges
```sql
SELECT DestCityName, uniqExact(OriginCityName) AS u F
ROM ontime
WHERE Year>=2000 and Year<=2010
SELECT DestCityName, uniqExact(OriginCityName) AS u
FROM ontime
WHERE Year >= 2000 and Year <= 2010
GROUP BY DestCityName
ORDER BY u DESC
LIMIT 10;
ORDER BY u DESC LIMIT 10;
```
Q9.

View File

@ -898,6 +898,7 @@ Possible values:
- [uniq](../../query_language/agg_functions/reference.md#agg_function-uniq)
- [uniqCombined](../../query_language/agg_functions/reference.md#agg_function-uniqcombined)
- [uniqCombined64](../../query_language/agg_functions/reference.md#agg_function-uniqcombined64)
- [uniqHLL12](../../query_language/agg_functions/reference.md#agg_function-uniqhll12)
- [uniqExact](../../query_language/agg_functions/reference.md#agg_function-uniqexact)

View File

@ -757,4 +757,9 @@ If there were problems with mutating some parts, the following columns contain a
**latest_fail_reason** - The exception message that caused the most recent part mutation failure.
## system.disks {#system_tables-disks}
## system.storage_policies {#system_tables-storage_policies}
[Original article](https://clickhouse.yandex/docs/en/operations/system_tables/) <!--hide-->

View File

@ -558,7 +558,7 @@ Calculates the approximate number of different argument values.
uniqCombined(HLL_precision)(x[, ...])
```
The `uniqCombined` function is a good choice for calculating the number of different values, but keep in mind that the estimation error for large sets (200 million elements and more) will be larger than the theoretical value due to the poor hash function choice.
The `uniqCombined` function is a good choice for calculating the number of different values.
**Parameters**

View File

@ -355,6 +355,8 @@ Before downloading, the system checks if the partition exists and the table stru
Although the query is called `ALTER TABLE`, it does not change the table structure and does not immediately change the data available in the table.
#### MOVE PARTITION|PART {#alter_move-partition}
#### How To Set Partition Expression {#alter-how-to-specify-part-expr}
You can specify the partition expression in `ALTER ... PARTITION` queries in different ways:

View File

@ -57,6 +57,17 @@ There is currently no support for code points in the format `\uXXXX\uYYYY` that
The following functions are based on [simdjson](https://github.com/lemire/simdjson) designed for more complex JSON parsing requirements. The assumption 2 mentioned above still applies.
## isValidJSON(json)
Checks that passed string is a valid json.
Examples:
```sql
SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1
SELECT isValidJSON('not a json') = 0
```
## JSONHas(json[, indices_or_keys]...)
If the value exists in the JSON document, `1` will be returned.

View File

@ -777,6 +777,7 @@ load_balancing = first_or_random
- [uniq](../../query_language/agg_functions/reference.md#agg_function-uniq)
- [uniqCombined](../../query_language/agg_functions/reference.md#agg_function-uniqcombined)
- [uniqCombined64](../../query_language/agg_functions/reference.md#agg_function-uniqcombined64)
- [uniqHLL12](../../query_language/agg_functions/reference.md#agg_function-uniqhll12)
- [uniqExact](../../query_language/agg_functions/reference.md#agg_function-uniqexact)

View File

@ -568,7 +568,7 @@ uniq(x[, ...])
uniqCombined(HLL_precision)(x[, ...])
```
Функция `uniqCombined` — это хороший выбор для вычисления количества различных значений, однако стоит иметь в виду, что ошибка оценки для больших множеств (более 200 миллионов элементов) будет выше теоретического значения из-за плохого выбора хэш-функции.
Функция `uniqCombined` — это хороший выбор для вычисления количества различных значений.
**Параметры**

View File

@ -57,6 +57,17 @@ visitParamExtractString('{"abc":"hello}', 'abc') = ''
Следующие функции используют [simdjson](https://github.com/lemire/simdjson) который разработан по более сложны требования для разбора JSON. Упомянутое выше предположение 2 по-прежнему применимо.
## isValidJSON(json)
Проверяет, является ли переданная строка валидным json значением.
Примеры:
```sql
SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1
SELECT isValidJSON('not a json') = 0
```
## JSONHas(json[, indices_or_keys]...)
Если значение существует в документе JSON, то возвращается `1`.

View File

@ -171,47 +171,57 @@ obstacle to adoption, that text has been removed.
#include <math.h>
#include <stdint.h>
#include <stdio.h>
double preciseExp10(double x)
{
if (isnan(x)) return NAN;
// ranging between DBL_TRUE_MIN and DBL_MAX. Outsiders are treated as zeros or infinities
static const double p10[]
= {1e-15, 1e-14, 1e-13, 1e-12, 1e-11, 1e-10, 1e-9, 1e-8, 1e-7, 1e-6, 1e-5, 1e-4, 1e-3, 1e-2, 1e-1, 1,
1e+1, 1e+2, 1e+3, 1e+4, 1e+5, 1e+6, 1e+7, 1e+8, 1e+9, 1e+10, 1e+11, 1e+12, 1e+13, 1e+14, 1e+15, 1e+16,
1e+17, 1e+18, 1e+19, 1e+20, 1e+21, 1e+22, 1e+23, 1e+24, 1e+25, 1e+26, 1e+27, 1e+28, 1e+29, 1e+30, 1e+31, 1e+32,
1e+33, 1e+34, 1e+35, 1e+36, 1e+37, 1e+38, 1e+39, 1e+40, 1e+41, 1e+42, 1e+43, 1e+44, 1e+45, 1e+46, 1e+47, 1e+48,
1e+49, 1e+50, 1e+51, 1e+52, 1e+53, 1e+54, 1e+55, 1e+56, 1e+57, 1e+58, 1e+59, 1e+60, 1e+61, 1e+62, 1e+63, 1e+64,
1e+65, 1e+66, 1e+67, 1e+68, 1e+69, 1e+70, 1e+71, 1e+72, 1e+73, 1e+74, 1e+75, 1e+76, 1e+77, 1e+78, 1e+79, 1e+80,
1e+81, 1e+82, 1e+83, 1e+84, 1e+85, 1e+86, 1e+87, 1e+88, 1e+89, 1e+90, 1e+91, 1e+92, 1e+93, 1e+94, 1e+95, 1e+96,
1e+97, 1e+98, 1e+99, 1e+100, 1e+101, 1e+102, 1e+103, 1e+104, 1e+105, 1e+106, 1e+107, 1e+108, 1e+109, 1e+110, 1e+111, 1e+112,
1e+113, 1e+114, 1e+115, 1e+116, 1e+117, 1e+118, 1e+119, 1e+120, 1e+121, 1e+122, 1e+123, 1e+124, 1e+125, 1e+126, 1e+127, 1e+128,
1e+129, 1e+130, 1e+131, 1e+132, 1e+133, 1e+134, 1e+135, 1e+136, 1e+137, 1e+138, 1e+139, 1e+140, 1e+141, 1e+142, 1e+143, 1e+144,
1e+145, 1e+146, 1e+147, 1e+148, 1e+149, 1e+150, 1e+151, 1e+152, 1e+153, 1e+154, 1e+155, 1e+156, 1e+157, 1e+158, 1e+159, 1e+160,
1e+161, 1e+162, 1e+163, 1e+164, 1e+165, 1e+166, 1e+167, 1e+168, 1e+169, 1e+170, 1e+171, 1e+172, 1e+173, 1e+174, 1e+175, 1e+176,
1e+177, 1e+178, 1e+179, 1e+180, 1e+181, 1e+182, 1e+183, 1e+184, 1e+185, 1e+186, 1e+187, 1e+188, 1e+189, 1e+190, 1e+191, 1e+192,
1e+193, 1e+194, 1e+195, 1e+196, 1e+197, 1e+198, 1e+199, 1e+200, 1e+201, 1e+202, 1e+203, 1e+204, 1e+205, 1e+206, 1e+207, 1e+208,
1e+209, 1e+210, 1e+211, 1e+212, 1e+213, 1e+214, 1e+215, 1e+216, 1e+217, 1e+218, 1e+219, 1e+220, 1e+221, 1e+222, 1e+223, 1e+224,
1e+225, 1e+226, 1e+227, 1e+228, 1e+229, 1e+230, 1e+231, 1e+232, 1e+233, 1e+234, 1e+235, 1e+236, 1e+237, 1e+238, 1e+239, 1e+240,
1e+241, 1e+242, 1e+243, 1e+244, 1e+245, 1e+246, 1e+247, 1e+248, 1e+249, 1e+250, 1e+251, 1e+252, 1e+253, 1e+254, 1e+255, 1e+256,
1e+257, 1e+258, 1e+259, 1e+260, 1e+261, 1e+262, 1e+263, 1e+264, 1e+265, 1e+266, 1e+267, 1e+268, 1e+269, 1e+270, 1e+271, 1e+272,
1e+273, 1e+274, 1e+275, 1e+276, 1e+277, 1e+278, 1e+279, 1e+280, 1e+281, 1e+282, 1e+283, 1e+284, 1e+285, 1e+286, 1e+287, 1e+288,
1e+289, 1e+290, 1e+291, 1e+292, 1e+293, 1e+294, 1e+295, 1e+296, 1e+297, 1e+298, 1e+299, 1e+300, 1e+301, 1e+302, 1e+303, 1e+304,
1e+305, 1e+306, 1e+307, 1e+308};
= {1e-323, 1e-322, 1e-321, 1e-320, 1e-319, 1e-318, 1e-317, 1e-316, 1e-315, 1e-314, 1e-313, 1e-312, 1e-311, 1e-310, 1e-309, 1e-308, 1e-307,
1e-306, 1e-305, 1e-304, 1e-303, 1e-302, 1e-301, 1e-300, 1e-299, 1e-298, 1e-297, 1e-296, 1e-295, 1e-294, 1e-293, 1e-292, 1e-291, 1e-290,
1e-289, 1e-288, 1e-287, 1e-286, 1e-285, 1e-284, 1e-283, 1e-282, 1e-281, 1e-280, 1e-279, 1e-278, 1e-277, 1e-276, 1e-275, 1e-274, 1e-273,
1e-272, 1e-271, 1e-270, 1e-269, 1e-268, 1e-267, 1e-266, 1e-265, 1e-264, 1e-263, 1e-262, 1e-261, 1e-260, 1e-259, 1e-258, 1e-257, 1e-256,
1e-255, 1e-254, 1e-253, 1e-252, 1e-251, 1e-250, 1e-249, 1e-248, 1e-247, 1e-246, 1e-245, 1e-244, 1e-243, 1e-242, 1e-241, 1e-240, 1e-239,
1e-238, 1e-237, 1e-236, 1e-235, 1e-234, 1e-233, 1e-232, 1e-231, 1e-230, 1e-229, 1e-228, 1e-227, 1e-226, 1e-225, 1e-224, 1e-223, 1e-222,
1e-221, 1e-220, 1e-219, 1e-218, 1e-217, 1e-216, 1e-215, 1e-214, 1e-213, 1e-212, 1e-211, 1e-210, 1e-209, 1e-208, 1e-207, 1e-206, 1e-205,
1e-204, 1e-203, 1e-202, 1e-201, 1e-200, 1e-199, 1e-198, 1e-197, 1e-196, 1e-195, 1e-194, 1e-193, 1e-192, 1e-191, 1e-190, 1e-189, 1e-188,
1e-187, 1e-186, 1e-185, 1e-184, 1e-183, 1e-182, 1e-181, 1e-180, 1e-179, 1e-178, 1e-177, 1e-176, 1e-175, 1e-174, 1e-173, 1e-172, 1e-171,
1e-170, 1e-169, 1e-168, 1e-167, 1e-166, 1e-165, 1e-164, 1e-163, 1e-162, 1e-161, 1e-160, 1e-159, 1e-158, 1e-157, 1e-156, 1e-155, 1e-154,
1e-153, 1e-152, 1e-151, 1e-150, 1e-149, 1e-148, 1e-147, 1e-146, 1e-145, 1e-144, 1e-143, 1e-142, 1e-141, 1e-140, 1e-139, 1e-138, 1e-137,
1e-136, 1e-135, 1e-134, 1e-133, 1e-132, 1e-131, 1e-130, 1e-129, 1e-128, 1e-127, 1e-126, 1e-125, 1e-124, 1e-123, 1e-122, 1e-121, 1e-120,
1e-119, 1e-118, 1e-117, 1e-116, 1e-115, 1e-114, 1e-113, 1e-112, 1e-111, 1e-110, 1e-109, 1e-108, 1e-107, 1e-106, 1e-105, 1e-104, 1e-103,
1e-102, 1e-101, 1e-100, 1e-99, 1e-98, 1e-97, 1e-96, 1e-95, 1e-94, 1e-93, 1e-92, 1e-91, 1e-90, 1e-89, 1e-88, 1e-87, 1e-86,
1e-85, 1e-84, 1e-83, 1e-82, 1e-81, 1e-80, 1e-79, 1e-78, 1e-77, 1e-76, 1e-75, 1e-74, 1e-73, 1e-72, 1e-71, 1e-70, 1e-69,
1e-68, 1e-67, 1e-66, 1e-65, 1e-64, 1e-63, 1e-62, 1e-61, 1e-60, 1e-59, 1e-58, 1e-57, 1e-56, 1e-55, 1e-54, 1e-53, 1e-52,
1e-51, 1e-50, 1e-49, 1e-48, 1e-47, 1e-46, 1e-45, 1e-44, 1e-43, 1e-42, 1e-41, 1e-40, 1e-39, 1e-38, 1e-37, 1e-36, 1e-35,
1e-34, 1e-33, 1e-32, 1e-31, 1e-30, 1e-29, 1e-28, 1e-27, 1e-26, 1e-25, 1e-24, 1e-23, 1e-22, 1e-21, 1e-20, 1e-19, 1e-18,
1e-17, 1e-16, 1e-15, 1e-14, 1e-13, 1e-12, 1e-11, 1e-10, 1e-9, 1e-8, 1e-7, 1e-6, 1e-5, 1e-4, 1e-3, 1e-2, 1e-1,
1e0, 1e+1, 1e+2, 1e+3, 1e+4, 1e+5, 1e+6, 1e+7, 1e+8, 1e+9, 1e+10, 1e+11, 1e+12, 1e+13, 1e+14, 1e+15, 1e+16,
1e+17, 1e+18, 1e+19, 1e+20, 1e+21, 1e+22, 1e+23, 1e+24, 1e+25, 1e+26, 1e+27, 1e+28, 1e+29, 1e+30, 1e+31, 1e+32, 1e+33,
1e+34, 1e+35, 1e+36, 1e+37, 1e+38, 1e+39, 1e+40, 1e+41, 1e+42, 1e+43, 1e+44, 1e+45, 1e+46, 1e+47, 1e+48, 1e+49, 1e+50,
1e+51, 1e+52, 1e+53, 1e+54, 1e+55, 1e+56, 1e+57, 1e+58, 1e+59, 1e+60, 1e+61, 1e+62, 1e+63, 1e+64, 1e+65, 1e+66, 1e+67,
1e+68, 1e+69, 1e+70, 1e+71, 1e+72, 1e+73, 1e+74, 1e+75, 1e+76, 1e+77, 1e+78, 1e+79, 1e+80, 1e+81, 1e+82, 1e+83, 1e+84,
1e+85, 1e+86, 1e+87, 1e+88, 1e+89, 1e+90, 1e+91, 1e+92, 1e+93, 1e+94, 1e+95, 1e+96, 1e+97, 1e+98, 1e+99, 1e+100, 1e+101,
1e+102, 1e+103, 1e+104, 1e+105, 1e+106, 1e+107, 1e+108, 1e+109, 1e+110, 1e+111, 1e+112, 1e+113, 1e+114, 1e+115, 1e+116, 1e+117, 1e+118,
1e+119, 1e+120, 1e+121, 1e+122, 1e+123, 1e+124, 1e+125, 1e+126, 1e+127, 1e+128, 1e+129, 1e+130, 1e+131, 1e+132, 1e+133, 1e+134, 1e+135,
1e+136, 1e+137, 1e+138, 1e+139, 1e+140, 1e+141, 1e+142, 1e+143, 1e+144, 1e+145, 1e+146, 1e+147, 1e+148, 1e+149, 1e+150, 1e+151, 1e+152,
1e+153, 1e+154, 1e+155, 1e+156, 1e+157, 1e+158, 1e+159, 1e+160, 1e+161, 1e+162, 1e+163, 1e+164, 1e+165, 1e+166, 1e+167, 1e+168, 1e+169,
1e+170, 1e+171, 1e+172, 1e+173, 1e+174, 1e+175, 1e+176, 1e+177, 1e+178, 1e+179, 1e+180, 1e+181, 1e+182, 1e+183, 1e+184, 1e+185, 1e+186,
1e+187, 1e+188, 1e+189, 1e+190, 1e+191, 1e+192, 1e+193, 1e+194, 1e+195, 1e+196, 1e+197, 1e+198, 1e+199, 1e+200, 1e+201, 1e+202, 1e+203,
1e+204, 1e+205, 1e+206, 1e+207, 1e+208, 1e+209, 1e+210, 1e+211, 1e+212, 1e+213, 1e+214, 1e+215, 1e+216, 1e+217, 1e+218, 1e+219, 1e+220,
1e+221, 1e+222, 1e+223, 1e+224, 1e+225, 1e+226, 1e+227, 1e+228, 1e+229, 1e+230, 1e+231, 1e+232, 1e+233, 1e+234, 1e+235, 1e+236, 1e+237,
1e+238, 1e+239, 1e+240, 1e+241, 1e+242, 1e+243, 1e+244, 1e+245, 1e+246, 1e+247, 1e+248, 1e+249, 1e+250, 1e+251, 1e+252, 1e+253, 1e+254,
1e+255, 1e+256, 1e+257, 1e+258, 1e+259, 1e+260, 1e+261, 1e+262, 1e+263, 1e+264, 1e+265, 1e+266, 1e+267, 1e+268, 1e+269, 1e+270, 1e+271,
1e+272, 1e+273, 1e+274, 1e+275, 1e+276, 1e+277, 1e+278, 1e+279, 1e+280, 1e+281, 1e+282, 1e+283, 1e+284, 1e+285, 1e+286, 1e+287, 1e+288,
1e+289, 1e+290, 1e+291, 1e+292, 1e+293, 1e+294, 1e+295, 1e+296, 1e+297, 1e+298, 1e+299, 1e+300, 1e+301, 1e+302, 1e+303, 1e+304, 1e+305,
1e+306, 1e+307, 1e+308};
double n, y = modf(x, &n);
if (n > 308)
return x > 0 ? INFINITY : -INFINITY;
if (!y)
return p10[(int)n + 15];
if (n > 308) return INFINITY;
if (n < -323) return 0;
union
{
double f;
uint64_t i;
} u = {n};
if ((u.i >> 52 & 0x7ff) < 0x3ff + 4)
{
y = exp2(3.32192809488736234787031942948939 * y);
return y * p10[(int)n + 15];
}
return pow(10.0, x);
// Using lookup table based formula to get accurate results for integer arguments.
return exp2(3.32192809488736234787031942948939 * y) * p10[(int)n + 323];
}