Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2018-09-02 06:00:57 +03:00
commit fefcccb48e
86 changed files with 473 additions and 471 deletions

View File

@ -137,7 +137,6 @@ if (USE_INTERNAL_CAPNP_LIBRARY)
target_include_directories(${CAPNP_LIBRARY} PUBLIC $<BUILD_INTERFACE:${CMAKE_CURRENT_SOURCE_DIR}/capnproto/c++/src>)
endif ()
if (USE_INTERNAL_POCO_LIBRARY)
set (save_CMAKE_CXX_FLAGS ${CMAKE_CXX_FLAGS})
set (save_CMAKE_C_FLAGS ${CMAKE_C_FLAGS})
@ -166,3 +165,11 @@ if (USE_INTERNAL_LLVM_LIBRARY)
endif ()
add_subdirectory (llvm/llvm)
endif ()
if (USE_INTERNAL_GTEST_LIBRARY)
# Google Test from sources
add_subdirectory(${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest)
# avoid problems with <regexp.h>
target_compile_definitions (gtest INTERFACE GTEST_HAS_POSIX_RE=0)
target_include_directories (gtest SYSTEM INTERFACE ${ClickHouse_SOURCE_DIR}/contrib/googletest/include)
endif ()

View File

@ -27,6 +27,16 @@ if (NOT NO_WERROR)
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Werror")
endif ()
# Add some warnings that are not available even with -Wall -Wextra.
if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra-semi -Wcomma -Winconsistent-missing-destructor-override -Wunused-exception-parameter -Wshadow-uncaptured-local")
if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER 5)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wredundant-parens -Wzero-as-null-pointer-constant")
endif ()
endif ()
if (USE_DEBUG_HELPERS)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -include ${ClickHouse_SOURCE_DIR}/libs/libcommon/include/common/iostream_debug_helpers.h")
endif ()
@ -252,15 +262,7 @@ add_subdirectory (programs)
add_subdirectory (tests)
if (ENABLE_TESTS)
if (USE_INTERNAL_GTEST_LIBRARY)
# Google Test from sources
add_subdirectory(${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest)
# avoid problems with <regexp.h>
target_compile_definitions (gtest INTERFACE GTEST_HAS_POSIX_RE=0)
target_include_directories (gtest SYSTEM INTERFACE ${ClickHouse_SOURCE_DIR}/contrib/googletest/include)
endif ()
macro(grep_gtest_sources BASE_DIR DST_VAR)
macro (grep_gtest_sources BASE_DIR DST_VAR)
# Cold match files that are not in tests/ directories
file(GLOB_RECURSE "${DST_VAR}" RELATIVE "${BASE_DIR}" "gtest*.cpp")
endmacro()

View File

@ -557,7 +557,7 @@ private:
fd_set fds;
FD_ZERO(&fds);
FD_SET(STDIN_FILENO, &fds);
return select(1, &fds, 0, 0, &timeout) == 1;
return select(1, &fds, nullptr, nullptr, &timeout) == 1;
}
inline const String prompt() const

View File

@ -1990,7 +1990,7 @@ protected:
if (increment_and_check_exit())
return;
}
catch (const Exception & e)
catch (const Exception &)
{
LOG_INFO(log, getCurrentExceptionMessage(false, true));
}

View File

@ -16,7 +16,6 @@ class Context;
class LocalServer : public Poco::Util::Application
{
public:
LocalServer();
void initialize(Poco::Util::Application & self) override;
@ -25,10 +24,9 @@ public:
void init(int argc, char ** argv);
~LocalServer();
~LocalServer() override;
private:
/** Composes CREATE subquery based on passed arguments (--structure --file --table and --input-format)
* This query will be executed first, before queries passed through --query argument
* Returns empty string if it cannot compose that query.
@ -46,7 +44,6 @@ private:
std::string getHelpFooter() const;
protected:
std::unique_ptr<Context> context;
/// Settings specified via command line args

View File

@ -123,7 +123,7 @@ int printHelp(int, char **)
for (auto & application : clickhouse_applications)
std::cerr << "clickhouse " << application.first << " [args] " << std::endl;
return -1;
};
}
bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)

View File

@ -127,7 +127,7 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
/// TODO Why 301?
SQLCHAR column_name[301];
/// TODO Result is not checked.
Poco::Data::ODBC::SQLDescribeCol(hstmt, ncol, column_name, sizeof(column_name), NULL, &type, NULL, NULL, NULL);
Poco::Data::ODBC::SQLDescribeCol(hstmt, ncol, column_name, sizeof(column_name), nullptr, &type, nullptr, nullptr, nullptr);
columns.emplace_back(reinterpret_cast<char *>(column_name), getDataType(type));
}

View File

@ -66,7 +66,7 @@ namespace
socket.listen(/* backlog = */ 64);
return address;
};
}
}
void ODBCBridge::handleHelp(const std::string &, const std::string &)

View File

@ -36,7 +36,7 @@ inline AggregateFunctionPtr createAggregateFunctionGroupArrayImpl(const DataType
return std::make_shared<GroupArrayGeneralListImpl<GroupArrayListNodeString, has_limit::value>>(argument_type, std::forward<TArgs>(args)...);
return std::make_shared<GroupArrayGeneralListImpl<GroupArrayListNodeGeneral, has_limit::value>>(argument_type, std::forward<TArgs>(args)...);
};
}
static AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name, const DataTypes & argument_types, const Array & parameters)

View File

@ -109,7 +109,6 @@ void ColumnAggregateFunction::insertRangeFrom(const IColumn & from, size_t start
/// Keep shared ownership of aggregation states.
src = from_concrete.getPtr();
auto & data = getData();
size_t old_size = data.size();
data.resize(old_size + length);
memcpy(&data[old_size], &from_concrete.getData()[start], length * sizeof(data[0]));
@ -182,7 +181,7 @@ ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray<Type> & indexe
return res;
}
INSTANTIATE_INDEX_IMPL(ColumnAggregateFunction);
INSTANTIATE_INDEX_IMPL(ColumnAggregateFunction)
/// Is required to support operations with Set
void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) const
@ -249,13 +248,13 @@ void ColumnAggregateFunction::insertData(const char * pos, size_t /*length*/)
getData().push_back(*reinterpret_cast<const AggregateDataPtr *>(pos));
}
void ColumnAggregateFunction::insertFrom(const IColumn & src, size_t n)
void ColumnAggregateFunction::insertFrom(const IColumn & from, size_t n)
{
/// Must create new state of aggregate function and take ownership of it,
/// because ownership of states of aggregate function cannot be shared for individual rows,
/// (only as a whole, see comment above).
insertDefault();
insertMergeFrom(src, n);
insertMergeFrom(from, n);
}
void ColumnAggregateFunction::insertFrom(ConstAggregateDataPtr place)
@ -269,9 +268,9 @@ void ColumnAggregateFunction::insertMergeFrom(ConstAggregateDataPtr place)
func->merge(getData().back(), place, &createOrGetArena());
}
void ColumnAggregateFunction::insertMergeFrom(const IColumn & src, size_t n)
void ColumnAggregateFunction::insertMergeFrom(const IColumn & from, size_t n)
{
insertMergeFrom(static_cast<const ColumnAggregateFunction &>(src).getData()[n]);
insertMergeFrom(static_cast<const ColumnAggregateFunction &>(from).getData()[n]);
}
Arena & ColumnAggregateFunction::createOrGetArena()

View File

@ -123,14 +123,14 @@ public:
void insertData(const char * pos, size_t length) override;
void insertFrom(const IColumn & src, size_t n) override;
void insertFrom(const IColumn & from, size_t n) override;
void insertFrom(ConstAggregateDataPtr place);
/// Merge state at last row with specified state in another column.
void insertMergeFrom(ConstAggregateDataPtr place);
void insertMergeFrom(const IColumn & src, size_t n);
void insertMergeFrom(const IColumn & from, size_t n);
Arena & createOrGetArena();

View File

@ -237,11 +237,11 @@ void ColumnArray::insertDefault()
void ColumnArray::popBack(size_t n)
{
auto & offsets = getOffsets();
size_t nested_n = offsets.back() - offsetAt(offsets.size() - n);
auto & offsets_data = getOffsets();
size_t nested_n = offsets_data.back() - offsetAt(offsets_data.size() - n);
if (nested_n)
getData().popBack(nested_n);
offsets.resize_assume_reserved(offsets.size() - n);
offsets_data.resize_assume_reserved(offsets_data.size() - n);
}
@ -662,7 +662,7 @@ ColumnPtr ColumnArray::indexImpl(const PaddedPODArray<T> & indexes, size_t limit
return res;
}
INSTANTIATE_INDEX_IMPL(ColumnArray);
INSTANTIATE_INDEX_IMPL(ColumnArray)
void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const
{

View File

@ -366,28 +366,28 @@ void ColumnNullable::getExtremes(Field & min, Field & max) const
min = Null();
max = Null();
const auto & null_map = getNullMapData();
const auto & null_map_data = getNullMapData();
if (const auto col = typeid_cast<const ColumnInt8 *>(nested_column.get()))
getExtremesFromNullableContent<Int8>(*col, null_map, min, max);
else if (const auto col = typeid_cast<const ColumnInt16 *>(nested_column.get()))
getExtremesFromNullableContent<Int16>(*col, null_map, min, max);
else if (const auto col = typeid_cast<const ColumnInt32 *>(nested_column.get()))
getExtremesFromNullableContent<Int32>(*col, null_map, min, max);
else if (const auto col = typeid_cast<const ColumnInt64 *>(nested_column.get()))
getExtremesFromNullableContent<Int64>(*col, null_map, min, max);
else if (const auto col = typeid_cast<const ColumnUInt8 *>(nested_column.get()))
getExtremesFromNullableContent<UInt8>(*col, null_map, min, max);
else if (const auto col = typeid_cast<const ColumnUInt16 *>(nested_column.get()))
getExtremesFromNullableContent<UInt16>(*col, null_map, min, max);
else if (const auto col = typeid_cast<const ColumnUInt32 *>(nested_column.get()))
getExtremesFromNullableContent<UInt32>(*col, null_map, min, max);
else if (const auto col = typeid_cast<const ColumnUInt64 *>(nested_column.get()))
getExtremesFromNullableContent<UInt64>(*col, null_map, min, max);
else if (const auto col = typeid_cast<const ColumnFloat32 *>(nested_column.get()))
getExtremesFromNullableContent<Float32>(*col, null_map, min, max);
else if (const auto col = typeid_cast<const ColumnFloat64 *>(nested_column.get()))
getExtremesFromNullableContent<Float64>(*col, null_map, min, max);
if (const auto col_i8 = typeid_cast<const ColumnInt8 *>(nested_column.get()))
getExtremesFromNullableContent<Int8>(*col_i8, null_map_data, min, max);
else if (const auto col_i16 = typeid_cast<const ColumnInt16 *>(nested_column.get()))
getExtremesFromNullableContent<Int16>(*col_i16, null_map_data, min, max);
else if (const auto col_i32 = typeid_cast<const ColumnInt32 *>(nested_column.get()))
getExtremesFromNullableContent<Int32>(*col_i32, null_map_data, min, max);
else if (const auto col_i64 = typeid_cast<const ColumnInt64 *>(nested_column.get()))
getExtremesFromNullableContent<Int64>(*col_i64, null_map_data, min, max);
else if (const auto col_u8 = typeid_cast<const ColumnUInt8 *>(nested_column.get()))
getExtremesFromNullableContent<UInt8>(*col_u8, null_map_data, min, max);
else if (const auto col_u16 = typeid_cast<const ColumnUInt16 *>(nested_column.get()))
getExtremesFromNullableContent<UInt16>(*col_u16, null_map_data, min, max);
else if (const auto col_u32 = typeid_cast<const ColumnUInt32 *>(nested_column.get()))
getExtremesFromNullableContent<UInt32>(*col_u32, null_map_data, min, max);
else if (const auto col_u64 = typeid_cast<const ColumnUInt64 *>(nested_column.get()))
getExtremesFromNullableContent<UInt64>(*col_u64, null_map_data, min, max);
else if (const auto col_f32 = typeid_cast<const ColumnFloat32 *>(nested_column.get()))
getExtremesFromNullableContent<Float32>(*col_f32, null_map_data, min, max);
else if (const auto col_f64 = typeid_cast<const ColumnFloat64 *>(nested_column.get()))
getExtremesFromNullableContent<Float64>(*col_f64, null_map_data, min, max);
}

View File

@ -513,4 +513,4 @@ IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType>::uniqueInsertRangeWi
return indexes_with_overflow;
}
};
}

View File

@ -141,12 +141,10 @@ struct ConfigReloader::FileWithTimestamp
};
void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path)
void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path_to_add)
{
if (!path.empty() && Poco::File(path).exists())
{
files.emplace(path, Poco::File(path).getLastModified().epochTime());
}
if (!path_to_add.empty() && Poco::File(path_to_add).exists())
files.emplace(path_to_add, Poco::File(path_to_add).getLastModified().epochTime());
}
bool ConfigReloader::FilesChangesTracker::isDifferOrNewerThan(const FilesChangesTracker & rhs)

View File

@ -56,7 +56,7 @@ private:
{
std::set<FileWithTimestamp> files;
void addIfExists(const std::string & path);
void addIfExists(const std::string & path_to_add);
bool isDifferOrNewerThan(const FilesChangesTracker & rhs);
};

View File

@ -48,7 +48,7 @@ bool FileChecker::check() const
* `check` method is rarely called.
*/
Map local_map;
load(local_map);
load(local_map, files_info_path);
if (local_map.empty())
return true;
@ -78,7 +78,7 @@ void FileChecker::initialize()
if (initialized)
return;
load(map);
load(map, files_info_path);
initialized = true;
}
@ -125,14 +125,14 @@ void FileChecker::save() const
Poco::File(tmp_files_info_path).renameTo(files_info_path);
}
void FileChecker::load(Map & map) const
void FileChecker::load(Map & local_map, const std::string & path)
{
map.clear();
local_map.clear();
if (!Poco::File(files_info_path).exists())
if (!Poco::File(path).exists())
return;
ReadBufferFromFile in(files_info_path);
ReadBufferFromFile in(path);
WriteBufferFromOwnString out;
/// The JSON library does not support whitespace. We delete them. Inefficient.
@ -147,7 +147,7 @@ void FileChecker::load(Map & map) const
JSON files = json["yandex"];
for (const auto & name_value : files)
map[unescapeForFileName(name_value.getName())] = name_value.getValue()["size"].toUInt();
local_map[unescapeForFileName(name_value.getName())] = name_value.getValue()["size"].toUInt();
}
}

View File

@ -30,7 +30,7 @@ private:
void initialize();
void updateImpl(const Poco::File & file);
void save() const;
void load(Map & map) const;
static void load(Map & local_map, const std::string & path);
std::string files_info_path;
std::string tmp_files_info_path;

View File

@ -227,10 +227,10 @@ private:
struct Cell
{
bool expired(const Timestamp & last_timestamp, const Delay & expiration_delay) const
bool expired(const Timestamp & last_timestamp, const Delay & delay) const
{
return (expiration_delay == Delay::zero()) ||
((last_timestamp > timestamp) && ((last_timestamp - timestamp) > expiration_delay));
return (delay == Delay::zero()) ||
((last_timestamp > timestamp) && ((last_timestamp - timestamp) > delay));
}
MappedPtr value;

View File

@ -172,11 +172,11 @@ RWLockFIFO::LockHandlerImpl::~LockHandlerImpl()
/// Remove the group if we were the last client and notify the next group
if (it_group->clients.empty())
{
auto & queue = parent->queue;
queue.erase(it_group);
auto & parent_queue = parent->queue;
parent_queue.erase(it_group);
if (!queue.empty())
queue.front().cv.notify_all();
if (!parent_queue.empty())
parent_queue.front().cv.notify_all();
}
parent.reset();

View File

@ -182,7 +182,8 @@ public:
{
/// @note assuming sequences for lowercase and uppercase have exact same length
const auto len = UTF8::seqLength(*pos);
pos += len, needle_pos += len;
pos += len;
needle_pos += len;
}
if (needle_pos == needle_end)
@ -206,7 +207,8 @@ public:
Poco::Unicode::toLower(utf8.convert(needle_pos)))
{
const auto len = UTF8::seqLength(*pos);
pos += len, needle_pos += len;
pos += len;
needle_pos += len;
}
if (needle_pos == needle_end)
@ -266,7 +268,8 @@ public:
{
/// @note assuming sequences for lowercase and uppercase have exact same length
const auto len = UTF8::seqLength(*haystack_pos);
haystack_pos += len, needle_pos += len;
haystack_pos += len;
needle_pos += len;
}
if (needle_pos == needle_end)
@ -296,7 +299,8 @@ public:
Poco::Unicode::toLower(utf8.convert(needle_pos)))
{
const auto len = UTF8::seqLength(*haystack_pos);
haystack_pos += len, needle_pos += len;
haystack_pos += len;
needle_pos += len;
}
if (needle_pos == needle_end)
@ -389,7 +393,10 @@ public:
auto needle_pos = needle + n;
while (needle_pos < needle_end && std::tolower(*pos) == std::tolower(*needle_pos))
++pos, ++needle_pos;
{
++pos;
++needle_pos;
}
if (needle_pos == needle_end)
return true;
@ -408,7 +415,10 @@ public:
auto needle_pos = needle + 1;
while (needle_pos < needle_end && std::tolower(*pos) == std::tolower(*needle_pos))
++pos, ++needle_pos;
{
++pos;
++needle_pos;
}
if (needle_pos == needle_end)
return true;
@ -460,7 +470,10 @@ public:
while (haystack_pos < haystack_end && needle_pos < needle_end &&
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
++haystack_pos, ++needle_pos;
{
++haystack_pos;
++needle_pos;
}
if (needle_pos == needle_end)
return haystack;
@ -485,7 +498,10 @@ public:
while (haystack_pos < haystack_end && needle_pos < needle_end &&
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
++haystack_pos, ++needle_pos;
{
++haystack_pos;
++needle_pos;
}
if (needle_pos == needle_end)
return haystack;

View File

@ -674,24 +674,24 @@ struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest
for (const auto & generic_request : generic_requests)
{
if (auto * concrete_request = dynamic_cast<const CreateRequest *>(generic_request.get()))
if (auto * concrete_request_create = dynamic_cast<const CreateRequest *>(generic_request.get()))
{
auto create = std::make_shared<ZooKeeperCreateRequest>(*concrete_request);
auto create = std::make_shared<ZooKeeperCreateRequest>(*concrete_request_create);
if (create->acls.empty())
create->acls = default_acls;
requests.push_back(create);
}
else if (auto * concrete_request = dynamic_cast<const RemoveRequest *>(generic_request.get()))
else if (auto * concrete_request_remove = dynamic_cast<const RemoveRequest *>(generic_request.get()))
{
requests.push_back(std::make_shared<ZooKeeperRemoveRequest>(*concrete_request));
requests.push_back(std::make_shared<ZooKeeperRemoveRequest>(*concrete_request_remove));
}
else if (auto * concrete_request = dynamic_cast<const SetRequest *>(generic_request.get()))
else if (auto * concrete_request_set = dynamic_cast<const SetRequest *>(generic_request.get()))
{
requests.push_back(std::make_shared<ZooKeeperSetRequest>(*concrete_request));
requests.push_back(std::make_shared<ZooKeeperSetRequest>(*concrete_request_set));
}
else if (auto * concrete_request = dynamic_cast<const CheckRequest *>(generic_request.get()))
else if (auto * concrete_request_check = dynamic_cast<const CheckRequest *>(generic_request.get()))
{
requests.push_back(std::make_shared<ZooKeeperCheckRequest>(*concrete_request));
requests.push_back(std::make_shared<ZooKeeperCheckRequest>(*concrete_request_check));
}
else
throw Exception("Illegal command as part of multi ZooKeeper request", ZBADARGUMENTS);
@ -914,11 +914,11 @@ void ZooKeeper::connect(
connected = true;
break;
}
catch (const Poco::Net::NetException & e)
catch (const Poco::Net::NetException &)
{
fail_reasons << "\n" << getCurrentExceptionMessage(false) << ", " << address.toString();
}
catch (const Poco::TimeoutException & e)
catch (const Poco::TimeoutException &)
{
fail_reasons << "\n" << getCurrentExceptionMessage(false);
}
@ -930,20 +930,20 @@ void ZooKeeper::connect(
if (!connected)
{
WriteBufferFromOwnString out;
out << "All connection tries failed while connecting to ZooKeeper. Addresses: ";
WriteBufferFromOwnString message;
message << "All connection tries failed while connecting to ZooKeeper. Addresses: ";
bool first = true;
for (const auto & address : addresses)
{
if (first)
first = false;
else
out << ", ";
out << address.toString();
message << ", ";
message << address.toString();
}
out << fail_reasons.str() << "\n";
throw Exception(out.str(), ZCONNECTIONLOSS);
message << fail_reasons.str() << "\n";
throw Exception(message.str(), ZCONNECTIONLOSS);
}
}
@ -953,7 +953,7 @@ void ZooKeeper::sendHandshake()
int32_t handshake_length = 44;
int64_t last_zxid_seen = 0;
int32_t timeout = session_timeout.totalMilliseconds();
int64_t session_id = 0;
int64_t previous_session_id = 0; /// We don't support session restore. So previous session_id is always zero.
constexpr int32_t passwd_len = 16;
std::array<char, passwd_len> passwd {};
@ -961,7 +961,7 @@ void ZooKeeper::sendHandshake()
write(protocol_version);
write(last_zxid_seen);
write(timeout);
write(session_id);
write(previous_session_id);
write(passwd);
out->next();
@ -1003,18 +1003,18 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data)
request.write(*out);
int32_t length;
XID xid;
XID read_xid;
int64_t zxid;
int32_t err;
read(length);
size_t count_before_event = in->count();
read(xid);
read(read_xid);
read(zxid);
read(err);
if (xid != auth_xid)
throw Exception("Unexpected event recieved in reply to auth request: " + toString(xid),
if (read_xid != auth_xid)
throw Exception("Unexpected event recieved in reply to auth request: " + toString(read_xid),
ZMARSHALLINGERROR);
int32_t actual_length = in->count() - count_before_event;
@ -1434,7 +1434,7 @@ void ZooKeeper::pushRequest(RequestInfo && info)
if (!info.request->xid)
{
info.request->xid = xid.fetch_add(1);
info.request->xid = next_xid.fetch_add(1);
if (info.request->xid < 0)
throw Exception("XID overflow", ZSESSIONEXPIRED);
}

View File

@ -111,7 +111,7 @@ public:
Poco::Timespan connection_timeout,
Poco::Timespan operation_timeout);
~ZooKeeper();
~ZooKeeper() override;
/// If expired, you can only destroy the object. All other methods will throw exception.
@ -179,7 +179,7 @@ private:
int64_t session_id = 0;
std::atomic<XID> xid {1};
std::atomic<XID> next_xid {1};
std::atomic<bool> expired {false};
std::mutex push_request_mutex;

View File

@ -32,10 +32,10 @@ std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
if (!zookeeper)
throw DB::Exception("Could not get znode: `" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_ZOOKEEPER);
for (const auto & path : invalidated_paths)
for (const auto & invalidated_path : invalidated_paths)
{
nonexistent_nodes.erase(path);
node_cache.erase(path);
nonexistent_nodes.erase(invalidated_path);
node_cache.erase(invalidated_path);
}
if (nonexistent_nodes.count(path))

View File

@ -5,14 +5,15 @@
#include <iostream>
#include <chrono>
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <gtest/gtest.h>
#include <Common/ShellCommand.h>
#pragma GCC diagnostic pop
using namespace DB;
TEST(zkutil, zookeeper_connected)

View File

@ -57,7 +57,7 @@ int main(int argc, char ** argv)
sleep(1);
}
}
catch (Coordination::Exception & e)
catch (Coordination::Exception &)
{
std::cerr << "KeeperException: " << DB::getCurrentExceptionMessage(true) << std::endl;
return 1;

View File

@ -210,13 +210,13 @@ void BicTest3(pfHash hash, const int reps, bool verbose = true)
for (int keybit = 0; keybit < keybits; keybit++)
{
int * page = &bins[keybit * pagesize];
int * bins = &page[(out1 * hashbits + out2) * 4];
int * bins_in_page = &page[(out1 * hashbits + out2) * 4];
double bias = 0;
for (int b = 0; b < 4; b++)
{
double b2 = static_cast<double>(bins[b]) / static_cast<double>(reps / 2);
double b2 = static_cast<double>(bins_in_page[b]) / static_cast<double>(reps / 2);
b2 = fabs(b2 * 2 - 1);
if (b2 > bias)

View File

@ -133,13 +133,14 @@ int main(int argc, char ** argv)
}
}
size_t n = 5;
size_t map_size = 1000000;
using T = DB::Field;
T field = std::string("Hello, world");
if (argc == 2 && !strcmp(argv[1], "1"))
{
size_t n = 5;
size_t map_size = 1000000;
using T = DB::Field;
T field = std::string("Hello, world");
using Arr = std::vector<T>;
using Map = HashMap<UInt64, Arr>;

View File

@ -1,7 +1,8 @@
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <gtest/gtest.h>
#pragma GCC diagnostic pop
#include <Common/RWLockFIFO.h>
#include <Common/Stopwatch.h>

View File

@ -1,9 +1,11 @@
#include <Common/escapeForFileName.h>
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <gtest/gtest.h>
#pragma GCC diagnostic pop
using namespace DB;

View File

@ -274,14 +274,11 @@ static inline void test(size_t n, const UInt64 * data, const char * name)
int main(int argc, char ** argv)
{
const size_t BUF_SIZE = 1024;
size_t n = (atoi(argv[1]) + (BUF_SIZE - 1)) / BUF_SIZE * BUF_SIZE;
size_t method = argc <= 2 ? 0 : atoi(argv[2]);
std::cerr << std::fixed << std::setprecision(2);
using Source = std::vector<UInt64>;
Source data(BUF_SIZE);
{

View File

@ -123,9 +123,9 @@ public:
bool isComparable() const override { return dictionary_type->isComparable(); }
bool canBeComparedWithCollation() const override { return dictionary_type->canBeComparedWithCollation(); }
bool canBeUsedAsVersion() const override { return dictionary_type->canBeUsedAsVersion(); }
bool isSummable() const override { return dictionary_type->isSummable(); };
bool canBeUsedInBitOperations() const override { return dictionary_type->canBeUsedInBitOperations(); };
bool canBeUsedInBooleanContext() const override { return dictionary_type->canBeUsedInBooleanContext(); };
bool isSummable() const override { return dictionary_type->isSummable(); }
bool canBeUsedInBitOperations() const override { return dictionary_type->canBeUsedInBitOperations(); }
bool canBeUsedInBooleanContext() const override { return dictionary_type->canBeUsedInBooleanContext(); }
bool isNumber() const override { return false; }
bool isInteger() const override { return false; }
bool isUnsignedInteger() const override { return false; }

View File

@ -4,11 +4,12 @@
#include <sstream>
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <gtest/gtest.h>
#pragma GCC diagnostic pop
using namespace DB;

View File

@ -44,23 +44,23 @@ private:
template <typename AttributeType>
ColumnPtr getColumnFromAttribute(DictionaryGetter<AttributeType> getter,
const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const;
ColumnPtr getColumnFromAttributeString(const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const;
template <typename T>
ColumnPtr getColumnFromPODArray(const PaddedPODArray<T> & array) const;
template <typename T>
void addSpecialColumn(
const std::optional<DictionarySpecialAttribute> & attribute, DataTypePtr type,
const std::string & default_name, const std::unordered_set<std::string> & column_names,
const std::string & default_name, const std::unordered_set<std::string> & column_names_set,
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const;
Block fillBlock(const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const;
const PaddedPODArray<UInt16> & block_start_dates, const PaddedPODArray<UInt16> & block_end_dates) const;
PaddedPODArray<UInt16> makeDateKey(
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const;
const PaddedPODArray<UInt16> & block_start_dates, const PaddedPODArray<UInt16> & block_end_dates) const;
DictionaryPtr dictionary;
Names column_names;
@ -104,20 +104,20 @@ template <typename DictionaryType, typename Key>
template <typename AttributeType>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
DictionaryGetter<AttributeType> getter, const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<UInt16> & dates, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const
const PaddedPODArray<UInt16> & dates, const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const
{
auto column_vector = ColumnVector<AttributeType>::create(ids_to_fill.size());
(dictionary.*getter)(attribute.name, ids_to_fill, dates, column_vector->getData());
(concrete_dictionary.*getter)(attribute.name, ids_to_fill, dates, column_vector->getData());
return column_vector;
}
template <typename DictionaryType, typename Key>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttributeString(
const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const
const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const
{
auto column_string = ColumnString::create();
dictionary.getString(attribute.name, ids_to_fill, dates, column_string.get());
concrete_dictionary.getString(attribute.name, ids_to_fill, dates, column_string.get());
return column_string;
}
@ -137,28 +137,28 @@ template <typename DictionaryType, typename Key>
template <typename T>
void RangeDictionaryBlockInputStream<DictionaryType, Key>::addSpecialColumn(
const std::optional<DictionarySpecialAttribute> & attribute, DataTypePtr type,
const std::string & default_name, const std::unordered_set<std::string> & column_names,
const std::string & default_name, const std::unordered_set<std::string> & column_names_set,
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const
{
std::string name = default_name;
if (attribute)
name = attribute->name;
if (column_names.find(name) != column_names.end())
if (column_names_set.find(name) != column_names_set.end())
columns.emplace_back(getColumnFromPODArray(values), type, name);
}
template <typename DictionaryType, typename Key>
PaddedPODArray<UInt16> RangeDictionaryBlockInputStream<DictionaryType, Key>::makeDateKey(
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const
const PaddedPODArray<UInt16> & block_start_dates, const PaddedPODArray<UInt16> & block_end_dates) const
{
PaddedPODArray<UInt16> key(start_dates.size());
PaddedPODArray<UInt16> key(block_start_dates.size());
for (size_t i = 0; i < key.size(); ++i)
{
if (RangeHashedDictionary::Range::isCorrectDate(start_dates[i]))
key[i] = start_dates[i];
if (RangeHashedDictionary::Range::isCorrectDate(block_start_dates[i]))
key[i] = block_start_dates[i];
else
key[i] = end_dates[i];
key[i] = block_end_dates[i];
}
return key;
@ -168,7 +168,7 @@ PaddedPODArray<UInt16> RangeDictionaryBlockInputStream<DictionaryType, Key>::mak
template <typename DictionaryType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const
const PaddedPODArray<UInt16> & block_start_dates, const PaddedPODArray<UInt16> & block_end_dates) const
{
ColumnsWithTypeAndName columns;
const DictionaryStructure & structure = dictionary->getStructure();
@ -176,10 +176,10 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
std::unordered_set<std::string> names(column_names.begin(), column_names.end());
addSpecialColumn(structure.id, std::make_shared<DataTypeUInt64>(), "ID", names, ids_to_fill, columns);
addSpecialColumn(structure.range_min, std::make_shared<DataTypeDate>(), "Range Start", names, start_dates, columns);
addSpecialColumn(structure.range_max, std::make_shared<DataTypeDate>(), "Range End", names, end_dates, columns);
addSpecialColumn(structure.range_min, std::make_shared<DataTypeDate>(), "Range Start", names, block_start_dates, columns);
addSpecialColumn(structure.range_max, std::make_shared<DataTypeDate>(), "Range End", names, block_end_dates, columns);
auto date_key = makeDateKey(start_dates, end_dates);
auto date_key = makeDateKey(block_start_dates, block_end_dates);
for (const auto idx : ext::range(0, structure.attributes.size()))
{

View File

@ -14,9 +14,9 @@ namespace ErrorCodes
RangeHashedDictionary::RangeHashedDictionary(
const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const std::string & dictionary_name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime, bool require_nonempty)
: name{name}, dict_struct(dict_struct),
: dictionary_name{dictionary_name}, dict_struct(dict_struct),
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
require_nonempty(require_nonempty)
{
@ -36,7 +36,7 @@ RangeHashedDictionary::RangeHashedDictionary(
}
RangeHashedDictionary::RangeHashedDictionary(const RangeHashedDictionary & other)
: RangeHashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty}
: RangeHashedDictionary{other.dictionary_name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty}
{
}
@ -102,7 +102,7 @@ void RangeHashedDictionary::createAttributes()
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
throw Exception{name + ": hierarchical attributes not supported by " + getName() + " dictionary.", ErrorCodes::BAD_ARGUMENTS};
throw Exception{dictionary_name + ": hierarchical attributes not supported by " + getName() + " dictionary.", ErrorCodes::BAD_ARGUMENTS};
}
}
@ -134,7 +134,7 @@ void RangeHashedDictionary::loadData()
stream->readSuffix();
if (require_nonempty && 0 == element_count)
throw Exception{name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY};
throw Exception{dictionary_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY};
}
template <typename T>
@ -280,9 +280,9 @@ void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, const K
auto & values = it->second;
const auto insert_it = std::lower_bound(std::begin(values), std::end(values), range,
[] (const Value<T> & lhs, const Range & range)
[] (const Value<T> & lhs, const Range & rhs_range)
{
return lhs.range < range;
return lhs.range < rhs_range;
});
values.insert(insert_it, Value<T>{ range, value });
@ -320,9 +320,9 @@ void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const Key i
auto & values = it->second;
const auto insert_it = std::lower_bound(std::begin(values), std::end(values), range,
[] (const Value<StringRef> & lhs, const Range & range)
[] (const Value<StringRef> & lhs, const Range & rhs_range)
{
return lhs.range < range;
return lhs.range < rhs_range;
});
values.insert(insert_it, Value<StringRef>{ range, string_ref });
@ -339,16 +339,16 @@ const RangeHashedDictionary::Attribute & RangeHashedDictionary::getAttribute(con
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
throw Exception{name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
throw Exception{dictionary_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
return attributes[it->second];
}
const RangeHashedDictionary::Attribute & RangeHashedDictionary::getAttributeWithType(const std::string & name, const AttributeUnderlyingType type) const
const RangeHashedDictionary::Attribute & RangeHashedDictionary::getAttributeWithType(const std::string & attribute_name, const AttributeUnderlyingType type) const
{
const auto & attribute = getAttribute(name);
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != type)
throw Exception{name + ": type mismatch: attribute " + name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};
throw Exception{attribute_name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};
return attribute;
}

View File

@ -18,14 +18,14 @@ class RangeHashedDictionary final : public IDictionaryBase
{
public:
RangeHashedDictionary(
const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const std::string & dictionary_name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime, bool require_nonempty);
RangeHashedDictionary(const RangeHashedDictionary & other);
std::exception_ptr getCreationException() const override { return creation_exception; }
std::string getName() const override { return name; }
std::string getName() const override { return dictionary_name; }
std::string getTypeName() const override { return "RangeHashed"; }
@ -180,7 +180,7 @@ private:
void getIdsAndDates(const Attribute & attribute, PaddedPODArray<Key> & ids,
PaddedPODArray<UInt16> & start_dates, PaddedPODArray<UInt16> & end_dates) const;
const std::string name;
const std::string dictionary_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;

View File

@ -1051,7 +1051,7 @@ class FunctionBinaryArithmetic : public IFunction
template <typename F>
static bool castBothTypes(const IDataType * left, const IDataType * right, F && f)
{
return castType(left, [&](const auto & left) { return castType(right, [&](const auto & right) { return f(left, right); }); });
return castType(left, [&](const auto & left_) { return castType(right, [&](const auto & right_) { return f(left_, right_); }); });
}
FunctionBuilderPtr getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1) const
@ -1188,9 +1188,9 @@ public:
return;
}
auto * left = block.getByPosition(arguments[0]).type.get();
auto * right = block.getByPosition(arguments[1]).type.get();
bool valid = castBothTypes(left, right, [&](const auto & left, const auto & right)
auto * left_generic = block.getByPosition(arguments[0]).type.get();
auto * right_generic = block.getByPosition(arguments[1]).type.get();
bool valid = castBothTypes(left_generic, right_generic, [&](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
@ -1265,7 +1265,7 @@ public:
auto & vec_res = col_res->getData();
vec_res.resize(block.rows());
if (auto col_left = checkAndGetColumnConst<ColVecT0>(col_left_raw))
if (auto col_left_const = checkAndGetColumnConst<ColVecT0>(col_left_raw))
{
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
{
@ -1278,13 +1278,13 @@ public:
if constexpr (IsDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
if (check_decimal_overflow)
OpImpl::constant_vector(col_left->template getValue<T0>(), col_right->getData(), vec_res, scale_a, scale_b);
OpImpl::constant_vector(col_left_const->template getValue<T0>(), col_right->getData(), vec_res, scale_a, scale_b);
else
OpImpl::XOverflow::constant_vector(
col_left->template getValue<T0>(), col_right->getData(), vec_res, scale_a, scale_b);
col_left_const->template getValue<T0>(), col_right->getData(), vec_res, scale_a, scale_b);
}
else
OpImpl::constant_vector(col_left->template getValue<T0>(), col_right->getData(), vec_res);
OpImpl::constant_vector(col_left_const->template getValue<T0>(), col_right->getData(), vec_res);
}
else
return false;
@ -1306,13 +1306,13 @@ public:
else
OpImpl::XOverflow::vector_vector(col_left->getData(), col_right->getData(), vec_res, scale_a, scale_b);
}
else if (auto col_right = checkAndGetColumnConst<ColVecT1>(col_right_raw))
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw))
{
if (check_decimal_overflow)
OpImpl::vector_constant(col_left->getData(), col_right->template getValue<T1>(), vec_res, scale_a, scale_b);
OpImpl::vector_constant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res, scale_a, scale_b);
else
OpImpl::XOverflow::vector_constant(
col_left->getData(), col_right->template getValue<T1>(), vec_res, scale_a, scale_b);
col_left->getData(), col_right_const->template getValue<T1>(), vec_res, scale_a, scale_b);
}
else
return false;
@ -1321,16 +1321,15 @@ public:
{
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
OpImpl::vector_vector(col_left->getData(), col_right->getData(), vec_res);
else if (auto col_right = checkAndGetColumnConst<ColVecT1>(col_right_raw))
OpImpl::vector_constant(col_left->getData(), col_right->template getValue<T1>(), vec_res);
else if (auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw))
OpImpl::vector_constant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res);
else
return false;
}
}
else
{
return false;
}
block.getByPosition(result).column = std::move(col_res);
return true;
}
@ -1858,7 +1857,7 @@ private:
{
const auto size = value_col->size();
bool is_const;
const auto mask = createConstMask<T>(block, arguments, is_const);
const auto const_mask = createConstMaskIfConst<T>(block, arguments, is_const);
const auto & val = value_col->getData();
auto out_col = ColumnVector<UInt8>::create(size);
@ -1867,7 +1866,7 @@ private:
if (is_const)
{
for (const auto i : ext::range(0, size))
out[i] = Impl::apply(val[i], mask);
out[i] = Impl::apply(val[i], const_mask);
}
else
{
@ -1880,16 +1879,16 @@ private:
block.getByPosition(result).column = std::move(out_col);
return true;
}
else if (const auto value_col = checkAndGetColumnConst<ColumnVector<T>>(value_col_untyped))
else if (const auto value_col_const = checkAndGetColumnConst<ColumnVector<T>>(value_col_untyped))
{
const auto size = value_col->size();
const auto size = value_col_const->size();
bool is_const;
const auto mask = createConstMask<T>(block, arguments, is_const);
const auto val = value_col->template getValue<T>();
const auto const_mask = createConstMaskIfConst<T>(block, arguments, is_const);
const auto val = value_col_const->template getValue<T>();
if (is_const)
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, toField(Impl::apply(val, mask)));
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, toField(Impl::apply(val, const_mask)));
}
else
{
@ -1911,9 +1910,9 @@ private:
}
template <typename ValueType>
ValueType createConstMask(const Block & block, const ColumnNumbers & arguments, bool & is_const)
ValueType createConstMaskIfConst(const Block & block, const ColumnNumbers & arguments, bool & out_is_const)
{
is_const = true;
out_is_const = true;
ValueType mask = 0;
for (const auto i : ext::range(1, arguments.size()))
@ -1925,7 +1924,7 @@ private:
}
else
{
is_const = false;
out_is_const = false;
return {};
}
}
@ -1964,9 +1963,9 @@ private:
return true;
}
else if (const auto pos_col = checkAndGetColumnConst<ColumnVector<PosType>>(pos_col_untyped))
else if (const auto pos_col_const = checkAndGetColumnConst<ColumnVector<PosType>>(pos_col_untyped))
{
const auto & pos = pos_col->template getValue<PosType>();
const auto & pos = pos_col_const->template getValue<PosType>();
const auto new_mask = 1 << pos;
for (const auto i : ext::range(0, mask.size()))

View File

@ -3575,7 +3575,7 @@ void FunctionArrayIntersect::NumberExecutor::operator()()
if (!result && typeid_cast<const DataTypeNumber<T> *>(data_type.get()))
result = execute<Map, ColumnVector<T>, true>(arrays, ColumnVector<T>::create());
};
}
template <typename Map, typename ColumnType, bool is_numeric_column>
ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, MutableColumnPtr result_data_ptr)

View File

@ -908,7 +908,7 @@ private:
const auto & value = (*item_arg)[row];
data[row] = 0;
for (size_t i = 0, size = arr.size(); i < size; ++i)
for (size_t i = 0, arr_size = arr.size(); i < arr_size; ++i)
{
bool hit = false;
@ -1466,7 +1466,7 @@ class FunctionArrayConcat : public IFunction
public:
static constexpr auto name = "arrayConcat";
static FunctionPtr create(const Context & context);
FunctionArrayConcat(const Context & context) : context(context) {};
FunctionArrayConcat(const Context & context) : context(context) {}
String getName() const override;
@ -1593,7 +1593,7 @@ class FunctionArrayIntersect : public IFunction
public:
static constexpr auto name = "arrayIntersect";
static FunctionPtr create(const Context & context);
FunctionArrayIntersect(const Context & context) : context(context) {};
FunctionArrayIntersect(const Context & context) : context(context) {}
String getName() const override;
@ -1695,7 +1695,7 @@ class FunctionArrayResize : public IFunction
public:
static constexpr auto name = "arrayResize";
static FunctionPtr create(const Context & context);
FunctionArrayResize(const Context & context) : context(context) {};
FunctionArrayResize(const Context & context) : context(context) {}
String getName() const override;

View File

@ -344,10 +344,10 @@ private:
if constexpr (_actual)
{
bool c0_const = c0->isColumnConst();
bool c1_const = c1->isColumnConst();
bool c0_is_const = c0->isColumnConst();
bool c1_is_const = c1->isColumnConst();
if (c0_const && c1_const)
if (c0_is_const && c1_is_const)
{
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
@ -361,7 +361,7 @@ private:
ColumnUInt8::Container & vec_res = c_res->getData();
vec_res.resize(c0->size());
if (c0_const)
if (c0_is_const)
{
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
A a = c0_const->template getValue<A>();
@ -370,7 +370,7 @@ private:
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
else if (c1_const)
else if (c1_is_const)
{
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
B b = c1_const->template getValue<B>();
@ -944,13 +944,13 @@ private:
block.getByPosition(result).column = std::move(col_res);
return true;
}
else if (auto col_right = checkAndGetColumnConst<ColumnVector<T1>>(col_right_untyped))
else if (auto col_right_const = checkAndGetColumnConst<ColumnVector<T1>>(col_right_untyped))
{
auto col_res = ColumnUInt8::create();
ColumnUInt8::Container & vec_res = col_res->getData();
vec_res.resize(col_left->size());
NumComparisonImpl<T0, T1, Op<T0, T1>>::vector_constant(col_left->getData(), col_right->template getValue<T1>(), vec_res);
NumComparisonImpl<T0, T1, Op<T0, T1>>::vector_constant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
@ -973,10 +973,10 @@ private:
block.getByPosition(result).column = std::move(col_res);
return true;
}
else if (auto col_right = checkAndGetColumnConst<ColumnVector<T1>>(col_right_untyped))
else if (auto col_right_const = checkAndGetColumnConst<ColumnVector<T1>>(col_right_untyped))
{
UInt8 res = 0;
NumComparisonImpl<T0, T1, Op<T0, T1>>::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>(), res);
NumComparisonImpl<T0, T1, Op<T0, T1>>::constant_constant(col_left->template getValue<T0>(), col_right_const->template getValue<T1>(), res);
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(col_left->size(), toField(res));
return true;
@ -1008,20 +1008,20 @@ private:
+ " of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
else if (auto col_left = checkAndGetColumnConst<ColumnVector<T0>>(col_left_untyped))
else if (auto col_left_const = checkAndGetColumnConst<ColumnVector<T0>>(col_left_untyped))
{
if ( executeNumConstRightType<T0, UInt8>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, UInt16>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, UInt32>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, UInt64>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, UInt128>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, Int8>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, Int16>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, Int32>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, Int64>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, Int128>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, Float32>(block, result, col_left, col_right_untyped)
|| executeNumConstRightType<T0, Float64>(block, result, col_left, col_right_untyped))
if ( executeNumConstRightType<T0, UInt8>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, UInt16>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, UInt32>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, UInt64>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, UInt128>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, Int8>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, Int16>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, Int32>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, Int64>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, Int128>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, Float32>(block, result, col_left_const, col_right_untyped)
|| executeNumConstRightType<T0, Float64>(block, result, col_left_const, col_right_untyped))
return true;
else
throw Exception("Illegal column " + col_right_untyped->getName()

View File

@ -92,7 +92,7 @@ public:
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionConsistentHashImpl<Impl>>();
};
}
String getName() const override
{

View File

@ -986,7 +986,7 @@ class FunctionToFixedString : public IFunction
{
public:
static constexpr auto name = "toFixedString";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToFixedString>(); };
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToFixedString>(); }
String getName() const override
{
@ -1625,7 +1625,7 @@ private:
throw Exception{"Enum conversion changes value for element '" + name_value.first +
"' from " + toString(old_value) + " to " + toString(new_value), ErrorCodes::CANNOT_CONVERT_TYPE};
}
};
}
template <typename ColumnStringType, typename EnumType>
WrapperType createStringToEnumWrapper() const

View File

@ -623,7 +623,7 @@ class FunctionDateOrDateTimeToSomething : public IFunction
{
public:
static constexpr auto name = Transform::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionDateOrDateTimeToSomething>(); };
static FunctionPtr create(const Context &) { return std::make_shared<FunctionDateOrDateTimeToSomething>(); }
String getName() const override
{
@ -928,7 +928,7 @@ class FunctionDateOrDateTimeAddInterval : public IFunction
{
public:
static constexpr auto name = Transform::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionDateOrDateTimeAddInterval>(); };
static FunctionPtr create(const Context &) { return std::make_shared<FunctionDateOrDateTimeAddInterval>(); }
String getName() const override
{
@ -1015,7 +1015,7 @@ class FunctionDateDiff : public IFunction
{
public:
static constexpr auto name = "dateDiff";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionDateDiff>(); };
static FunctionPtr create(const Context &) { return std::make_shared<FunctionDateDiff>(); }
String getName() const override
{
@ -1192,7 +1192,7 @@ class FunctionNow : public IFunction
{
public:
static constexpr auto name = "now";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionNow>(); };
static FunctionPtr create(const Context &) { return std::make_shared<FunctionNow>(); }
String getName() const override
{
@ -1221,7 +1221,7 @@ class FunctionToday : public IFunction
{
public:
static constexpr auto name = "today";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToday>(); };
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToday>(); }
String getName() const override
{
@ -1250,7 +1250,7 @@ class FunctionYesterday : public IFunction
{
public:
static constexpr auto name = "yesterday";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionYesterday>(); };
static FunctionPtr create(const Context &) { return std::make_shared<FunctionYesterday>(); }
String getName() const override
{
@ -1280,7 +1280,7 @@ class FunctionToTimeZone : public IFunction
{
public:
static constexpr auto name = "toTimeZone";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToTimeZone>(); };
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToTimeZone>(); }
String getName() const override
{
@ -1315,7 +1315,7 @@ class FunctionTimeSlot : public IFunction
{
public:
static constexpr auto name = "timeSlot";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionTimeSlot>(); };
static FunctionPtr create(const Context &) { return std::make_shared<FunctionTimeSlot>(); }
String getName() const override
{
@ -1442,7 +1442,7 @@ class FunctionTimeSlots : public IFunction
{
public:
static constexpr auto name = "timeSlots";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionTimeSlots>(); };
static FunctionPtr create(const Context &) { return std::make_shared<FunctionTimeSlots>(); }
String getName() const override
{

View File

@ -768,4 +768,4 @@ public:
}
};
};
}

View File

@ -1547,5 +1547,4 @@ private:
const ExternalDictionaries & dictionaries;
};
};
}

View File

@ -107,11 +107,11 @@ public:
{
DataTypes types;
types.reserve(names.size());
for (const auto & name : names)
for (const auto & captured_name : names)
{
auto it = arguments_map.find(name);
auto it = arguments_map.find(captured_name);
if (it == arguments_map.end())
throw Exception("Lambda captured argument " + name + " not found in required columns.",
throw Exception("Lambda captured argument " + captured_name + " not found in required columns.",
ErrorCodes::LOGICAL_ERROR);
types.push_back(it->second);

View File

@ -372,9 +372,15 @@ void LowerUpperUTF8Impl<not_case_lower_bound, not_case_upper_bound, to_case, cyr
static const Poco::UTF8Encoding utf8;
if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src))
src += chars, dst += chars;
{
src += chars;
dst += chars;
}
else
++src, ++dst;
{
++src;
++dst;
}
}
}
@ -426,7 +432,8 @@ void LowerUpperUTF8Impl<not_case_lower_bound, not_case_upper_bound, to_case, cyr
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars);
}
src += bytes_sse, dst += bytes_sse;
src += bytes_sse;
dst += bytes_sse;
}
else
{

View File

@ -47,7 +47,8 @@ template <bool>
UInt8 xor_or_identity(const UInt8 c, const int mask)
{
return c ^ mask;
};
}
template <>
inline UInt8 xor_or_identity<false>(const UInt8 c, const int)
{

View File

@ -745,7 +745,7 @@ private:
if (initialized)
return;
if (from.size() != to.size())
if (size != to.size())
throw Exception{"Second and third arguments of function " + getName() + " must be arrays of same size", ErrorCodes::BAD_ARGUMENTS};
Array converted_to;
@ -772,8 +772,8 @@ private:
if (default_col_is_float && !to_is_float)
{
converted_to.resize(to.size());
for (size_t i = 0, size = to.size(); i < size; ++i)
converted_to.resize(size);
for (size_t i = 0; i < size; ++i)
converted_to[i] = applyVisitor(FieldVisitorConvertToNumber<Float64>(), to[i]);
used_to = &converted_to;
}

View File

@ -84,13 +84,13 @@ void AIOContextPool::fulfillPromises(const io_event events[], const int num_even
for (const auto & event : boost::make_iterator_range(events, events + num_events))
{
/// get id from event
const auto id = event.data;
const auto completed_id = event.data;
/// set value via promise and release it
const auto it = promises.find(id);
const auto it = promises.find(completed_id);
if (it == std::end(promises))
{
LOG_ERROR(&Poco::Logger::get("AIOcontextPool"), "Found io_event with unknown id " << id);
LOG_ERROR(&Poco::Logger::get("AIOcontextPool"), "Found io_event with unknown id " << completed_id);
continue;
}
@ -126,7 +126,8 @@ std::future<AIOContextPool::BytesRead> AIOContextPool::post(struct iocb & iocb)
std::unique_lock<std::mutex> lock{mutex};
/// get current id and increment it by one
const auto request_id = id++;
const auto request_id = next_id;
++next_id;
/// create a promise and put request in "queue"
promises.emplace(request_id, std::promise<BytesRead>{});

View File

@ -26,7 +26,7 @@ class AIOContextPool : public ext::singleton<AIOContextPool>
using BytesRead = ssize_t;
/// Autoincremental id used to identify completed requests
ID id{};
ID next_id{};
mutable std::mutex mutex;
mutable std::condition_variable have_resources;
std::map<ID, std::promise<BytesRead>> promises;

View File

@ -189,7 +189,7 @@ off_t ReadBufferAIO::doSeek(off_t off, int whence)
void ReadBufferAIO::synchronousRead()
{
CurrentMetrics::Increment metric_increment{CurrentMetrics::Read};
CurrentMetrics::Increment metric_increment_read{CurrentMetrics::Read};
prepare();
bytes_read = ::pread(fd, buffer_begin, region_aligned_size, region_aligned_begin);
@ -225,7 +225,7 @@ bool ReadBufferAIO::waitForAIOCompletion()
if (is_eof || !is_pending_read)
return false;
CurrentMetrics::Increment metric_increment{CurrentMetrics::Read};
CurrentMetrics::Increment metric_increment_read{CurrentMetrics::Read};
bytes_read = future_bytes_read.get();
is_pending_read = false;

View File

@ -178,7 +178,7 @@ bool WriteBufferAIO::waitForAIOCompletion()
if (!is_pending_write)
return false;
CurrentMetrics::Increment metric_increment{CurrentMetrics::Write};
CurrentMetrics::Increment metric_increment_write{CurrentMetrics::Write};
io_event event;
while (io_getevents(aio_context.ctx, 1, 1, &event, nullptr) < 0)

View File

@ -14,9 +14,9 @@ ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer(
: BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment)
, out(out_)
{
zstr.zalloc = Z_NULL;
zstr.zfree = Z_NULL;
zstr.opaque = Z_NULL;
zstr.zalloc = nullptr;
zstr.zfree = nullptr;
zstr.opaque = nullptr;
zstr.next_in = nullptr;
zstr.avail_in = 0;
zstr.next_out = nullptr;

View File

@ -14,9 +14,9 @@ ZlibInflatingReadBuffer::ZlibInflatingReadBuffer(
, in(in_)
, eof(false)
{
zstr.zalloc = Z_NULL;
zstr.zfree = Z_NULL;
zstr.opaque = Z_NULL;
zstr.zalloc = nullptr;
zstr.zfree = nullptr;
zstr.opaque = nullptr;
zstr.next_in = nullptr;
zstr.avail_in = 0;
zstr.next_out = nullptr;
@ -70,7 +70,7 @@ bool ZlibInflatingReadBuffer::nextImpl()
}
else
{
int rc = inflateReset(&zstr);
rc = inflateReset(&zstr);
if (rc != Z_OK)
throw Exception(std::string("inflateReset failed: ") + zError(rc), ErrorCodes::ZLIB_INFLATE_FAILED);
return true;

View File

@ -372,7 +372,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
auto after_point_num_leading_zeros = after_leading_zeros_count - after_point_count;
readUIntTextUpToNSignificantDigits<significant_digits>(after_point, in);
int read_digits = in.count() - after_leading_zeros_count;
read_digits = in.count() - after_leading_zeros_count;
after_point_exponent = (read_digits > significant_digits ? -significant_digits : -read_digits) - after_point_num_leading_zeros;
}

View File

@ -1,7 +1,8 @@
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <gtest/gtest.h>
#pragma GCC diagnostic pop
#include <stdexcept>
#include <Poco/File.h>

View File

@ -25,7 +25,7 @@ try
return 0;
}
catch (const Exception & e)
catch (const Exception &)
{
std::cerr << getCurrentExceptionMessage(true) << std::endl;
return 1;

View File

@ -184,7 +184,7 @@ Aggregator::Aggregator(const Params & params_)
all_aggregates_has_trivial_destructor = false;
}
method = chooseAggregationMethod();
method_chosen = chooseAggregationMethod();
}
@ -200,14 +200,14 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
#if !defined(INTERNAL_COMPILER_HEADERS)
throw Exception("Cannot compile code: Compiler disabled", ErrorCodes::CANNOT_COMPILE_CODE);
#else
std::string method_typename;
std::string method_typename_single_level;
std::string method_typename_two_level;
if (false) {}
#define M(NAME) \
else if (type == AggregatedDataVariants::Type::NAME) \
{ \
method_typename = "decltype(AggregatedDataVariants::" #NAME ")::element_type"; \
method_typename_single_level = "decltype(AggregatedDataVariants::" #NAME ")::element_type"; \
method_typename_two_level = "decltype(AggregatedDataVariants::" #NAME "_two_level)::element_type"; \
}
@ -216,7 +216,7 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
#define M(NAME) \
else if (type == AggregatedDataVariants::Type::NAME) \
method_typename = "decltype(AggregatedDataVariants::" #NAME ")::element_type";
method_typename_single_level = "decltype(AggregatedDataVariants::" #NAME ")::element_type";
APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M)
#undef M
@ -260,12 +260,12 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
std::stringstream key_str;
key_str << "Aggregate: ";
if (!method_typename.empty())
key_str << method_typename + ", ";
if (!method_typename_single_level.empty())
key_str << method_typename_single_level + ", ";
key_str << aggregate_functions_typenames;
std::string key = key_str.str();
auto get_code = [method_typename, method_typename_two_level, aggregate_functions_typenames]
auto get_code = [method_typename_single_level, method_typename_two_level, aggregate_functions_typenames]
{
/// A short piece of code, which is an explicit instantiation of the template.
std::stringstream code;
@ -282,7 +282,7 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
"template void Aggregator::executeSpecialized<\n"
" " << method_typename << ", TypeList<" << aggregate_functions_typenames << ">>(\n"
" " << method_typename << " &, Arena *, size_t, ColumnRawPtrs &,\n"
" AggregateColumns &, const Sizes &, StringRefs &, bool, AggregateDataPtr) const;\n"
" AggregateColumns &, StringRefs &, bool, AggregateDataPtr) const;\n"
"\n"
"static void wrapper" << suffix << "(\n"
" const Aggregator & aggregator,\n"
@ -291,14 +291,13 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
" size_t rows,\n"
" ColumnRawPtrs & key_columns,\n"
" Aggregator::AggregateColumns & aggregate_columns,\n"
" const Sizes & key_sizes,\n"
" StringRefs & keys,\n"
" bool no_more_keys,\n"
" AggregateDataPtr overflow_row)\n"
"{\n"
" aggregator.executeSpecialized<\n"
" " << method_typename << ", TypeList<" << aggregate_functions_typenames << ">>(\n"
" method, arena, rows, key_columns, aggregate_columns, key_sizes, keys, no_more_keys, overflow_row);\n"
" method, arena, rows, key_columns, aggregate_columns, keys, no_more_keys, overflow_row);\n"
"}\n"
"\n"
"void * getPtr" << suffix << "() __attribute__((__visibility__(\"default\")));\n"
@ -308,8 +307,8 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
"}\n";
};
if (!method_typename.empty())
append_code_for_specialization(method_typename, "");
if (!method_typename_single_level.empty())
append_code_for_specialization(method_typename_single_level, "");
else
{
/// For `without_key` method.
@ -544,7 +543,6 @@ void NO_INLINE Aggregator::executeImpl(
size_t rows,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions,
const Sizes & key_sizes,
StringRefs & keys,
bool no_more_keys,
AggregateDataPtr overflow_row) const
@ -553,9 +551,9 @@ void NO_INLINE Aggregator::executeImpl(
state.init(key_columns);
if (!no_more_keys)
executeImplCase<false>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, key_sizes, keys, overflow_row);
executeImplCase<false>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, keys, overflow_row);
else
executeImplCase<true>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, key_sizes, keys, overflow_row);
executeImplCase<true>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, keys, overflow_row);
}
#ifndef __clang__
@ -571,7 +569,6 @@ void NO_INLINE Aggregator::executeImplCase(
size_t rows,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions,
const Sizes & key_sizes,
StringRefs & keys,
AggregateDataPtr overflow_row) const
{
@ -739,7 +736,7 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
/// How to perform the aggregation?
if (result.empty())
{
result.init(method);
result.init(method_chosen);
result.keys_size = params.keys_size;
result.key_sizes = key_sizes;
LOG_TRACE(log, "Aggregation method: " << result.getMethodName());
@ -788,9 +785,9 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
reinterpret_cast<void (*)( \
const Aggregator &, decltype(result.NAME)::element_type &, \
Arena *, size_t, ColumnRawPtrs &, AggregateColumns &, \
const Sizes &, StringRefs &, bool, AggregateDataPtr)>(compiled_data->compiled_method_ptr) \
StringRefs &, bool, AggregateDataPtr)>(compiled_data->compiled_method_ptr) \
(*this, *result.NAME, result.aggregates_pool, rows, key_columns, aggregate_columns, \
result.key_sizes, key, no_more_keys, overflow_row_ptr);
key, no_more_keys, overflow_row_ptr);
if (false) {}
APPLY_FOR_AGGREGATED_VARIANTS(M)
@ -804,9 +801,9 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
reinterpret_cast<void (*)( \
const Aggregator &, decltype(result.NAME)::element_type &, \
Arena *, size_t, ColumnRawPtrs &, AggregateColumns &, \
const Sizes &, StringRefs &, bool, AggregateDataPtr)>(compiled_data->compiled_two_level_method_ptr) \
StringRefs &, bool, AggregateDataPtr)>(compiled_data->compiled_two_level_method_ptr) \
(*this, *result.NAME, result.aggregates_pool, rows, key_columns, aggregate_columns, \
result.key_sizes, key, no_more_keys, overflow_row_ptr);
key, no_more_keys, overflow_row_ptr);
if (false) {}
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
@ -818,7 +815,7 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
executeImpl(*result.NAME, result.aggregates_pool, rows, key_columns, &aggregate_functions_instructions[0], \
result.key_sizes, key, no_more_keys, overflow_row_ptr);
key, no_more_keys, overflow_row_ptr);
if (false) {}
APPLY_FOR_AGGREGATED_VARIANTS(M)
@ -939,11 +936,10 @@ Block Aggregator::convertOneBucketToBlock(
MutableColumns & key_columns,
AggregateColumnsData & aggregate_columns,
MutableColumns & final_aggregate_columns,
const Sizes & key_sizes,
bool final)
bool final_)
{
convertToBlockImpl(method, method.data.impls[bucket],
key_columns, aggregate_columns, final_aggregate_columns, key_sizes, final);
key_columns, aggregate_columns, final_aggregate_columns, final_);
});
block.info.bucket_num = bucket;
@ -1076,16 +1072,15 @@ void Aggregator::convertToBlockImpl(
MutableColumns & key_columns,
AggregateColumnsData & aggregate_columns,
MutableColumns & final_aggregate_columns,
const Sizes & key_sizes,
bool final) const
{
if (data.empty())
return;
if (final)
convertToBlockImplFinal(method, data, key_columns, final_aggregate_columns, key_sizes);
convertToBlockImplFinal(method, data, key_columns, final_aggregate_columns);
else
convertToBlockImplNotFinal(method, data, key_columns, aggregate_columns, key_sizes);
convertToBlockImplNotFinal(method, data, key_columns, aggregate_columns);
/// In order to release memory early.
data.clearAndShrink();
@ -1097,8 +1092,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
Method & method,
Table & data,
MutableColumns & key_columns,
MutableColumns & final_aggregate_columns,
const Sizes & key_sizes) const
MutableColumns & final_aggregate_columns) const
{
for (const auto & value : data)
{
@ -1118,8 +1112,7 @@ void NO_INLINE Aggregator::convertToBlockImplNotFinal(
Method & method,
Table & data,
MutableColumns & key_columns,
AggregateColumnsData & aggregate_columns,
const Sizes & key_sizes) const
AggregateColumnsData & aggregate_columns) const
{
for (auto & value : data)
@ -1186,7 +1179,7 @@ Block Aggregator::prepareBlockAndFill(
}
}
filler(key_columns, aggregate_columns_data, final_aggregate_columns, data_variants.key_sizes, final);
filler(key_columns, aggregate_columns_data, final_aggregate_columns, final);
Block res = header.cloneEmpty();
@ -1219,8 +1212,7 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va
MutableColumns & key_columns,
AggregateColumnsData & aggregate_columns,
MutableColumns & final_aggregate_columns,
const Sizes & /*key_sizes*/,
bool final)
bool final_)
{
if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
{
@ -1228,13 +1220,13 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va
for (size_t i = 0; i < params.aggregates_size; ++i)
{
if (!final)
if (!final_)
aggregate_columns[i]->push_back(data + offsets_of_aggregate_states[i]);
else
aggregate_functions[i]->insertResultInto(data + offsets_of_aggregate_states[i], *final_aggregate_columns[i]);
}
if (!final)
if (!final_)
data = nullptr;
if (params.overflow_row)
@ -1262,13 +1254,12 @@ Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_v
MutableColumns & key_columns,
AggregateColumnsData & aggregate_columns,
MutableColumns & final_aggregate_columns,
const Sizes & /*key_sizes*/,
bool final)
bool final_)
{
#define M(NAME) \
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
convertToBlockImpl(*data_variants.NAME, data_variants.NAME->data, \
key_columns, aggregate_columns, final_aggregate_columns, data_variants.key_sizes, final);
key_columns, aggregate_columns, final_aggregate_columns, final_);
if (false) {}
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
@ -1525,11 +1516,11 @@ void NO_INLINE Aggregator::mergeWithoutKeyDataImpl(
{
AggregatedDataVariantsPtr & res = non_empty_data[0];
/// We connect all aggregation results to the first.
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
/// We merge all aggregation results to the first.
for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num)
{
AggregatedDataWithoutKey & res_data = res->without_key;
AggregatedDataWithoutKey & current_data = non_empty_data[i]->without_key;
AggregatedDataWithoutKey & current_data = non_empty_data[result_num]->without_key;
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge(res_data + offsets_of_aggregate_states[i], current_data + offsets_of_aggregate_states[i], res->aggregates_pool);
@ -1549,13 +1540,13 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl(
AggregatedDataVariantsPtr & res = non_empty_data[0];
bool no_more_keys = false;
/// We connect all aggregation results to the first.
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
/// We merge all aggregation results to the first.
for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num)
{
if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys))
break;
AggregatedDataVariants & current = *non_empty_data[i];
AggregatedDataVariants & current = *non_empty_data[result_num];
if (!no_more_keys)
mergeDataImpl<Method>(
@ -1584,11 +1575,11 @@ template <typename Method>
void NO_INLINE Aggregator::mergeBucketImpl(
ManyAggregatedDataVariants & data, Int32 bucket, Arena * arena) const
{
/// We connect all aggregation results to the first.
/// We merge all aggregation results to the first.
AggregatedDataVariantsPtr & res = data[0];
for (size_t i = 1, size = data.size(); i < size; ++i)
for (size_t result_num = 1, size = data.size(); result_num < size; ++result_num)
{
AggregatedDataVariants & current = *data[i];
AggregatedDataVariants & current = *data[result_num];
mergeDataImpl<Method>(
getDataVariant<Method>(*res).data.impls[bucket],
@ -1862,7 +1853,6 @@ std::unique_ptr<IBlockInputStream> Aggregator::mergeAndConvertToBlocks(
template <bool no_more_keys, typename Method, typename Table>
void NO_INLINE Aggregator::mergeStreamsImplCase(
Block & block,
const Sizes & key_sizes,
Arena * aggregates_pool,
Method & method,
Table & data,
@ -1945,7 +1935,6 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
template <typename Method, typename Table>
void NO_INLINE Aggregator::mergeStreamsImpl(
Block & block,
const Sizes & key_sizes,
Arena * aggregates_pool,
Method & method,
Table & data,
@ -1953,9 +1942,9 @@ void NO_INLINE Aggregator::mergeStreamsImpl(
bool no_more_keys) const
{
if (!no_more_keys)
mergeStreamsImplCase<false>(block, key_sizes, aggregates_pool, method, data, overflow_row);
mergeStreamsImplCase<false>(block, aggregates_pool, method, data, overflow_row);
else
mergeStreamsImplCase<true>(block, key_sizes, aggregates_pool, method, data, overflow_row);
mergeStreamsImplCase<true>(block, aggregates_pool, method, data, overflow_row);
}
@ -2029,8 +2018,8 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
if (has_two_level)
{
#define M(NAME) \
if (method == AggregatedDataVariants::Type::NAME) \
method = AggregatedDataVariants::Type::NAME ## _two_level;
if (method_chosen == AggregatedDataVariants::Type::NAME) \
method_chosen = AggregatedDataVariants::Type::NAME ## _two_level;
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
@ -2043,7 +2032,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
/// result will destroy the states of aggregate functions in the destructor
result.aggregator = this;
result.init(method);
result.init(method_chosen);
result.keys_size = params.keys_size;
result.key_sizes = key_sizes;
@ -2070,7 +2059,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
#define M(NAME) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(block, result.key_sizes, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket], nullptr, false);
mergeStreamsImpl(block, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket], nullptr, false);
if (false) {}
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
@ -2137,7 +2126,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(block, result.key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys);
mergeStreamsImpl(block, result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys);
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
@ -2165,7 +2154,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
* Better hash function is needed because during external aggregation,
* we may merge partitions of data with total number of keys far greater than 4 billion.
*/
auto merge_method = method;
auto merge_method = method_chosen;
#define APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION(M) \
M(key64) \
@ -2208,7 +2197,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data, nullptr, false);
mergeStreamsImpl(block, result.aggregates_pool, *result.NAME, result.NAME->data, nullptr, false);
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
@ -2254,7 +2243,6 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl(
Method & method,
Arena * pool,
ColumnRawPtrs & key_columns,
const Sizes & key_sizes,
StringRefs & keys,
const Block & source,
std::vector<Block> & destinations) const
@ -2321,7 +2309,7 @@ std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block)
for (size_t i = 0; i < params.keys_size; ++i)
key_columns[i] = block.safeGetByPosition(i).column.get();
AggregatedDataVariants::Type type = method;
AggregatedDataVariants::Type type = method_chosen;
data.keys_size = params.keys_size;
data.key_sizes = key_sizes;
@ -2354,7 +2342,7 @@ std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block)
#define M(NAME) \
else if (data.type == AggregatedDataVariants::Type::NAME) \
convertBlockToTwoLevelImpl(*data.NAME, data.aggregates_pool, \
key_columns, data.key_sizes, key, block, splitted_blocks);
key_columns, key, block, splitted_blocks);
if (false) {}
APPLY_FOR_VARIANTS_TWO_LEVEL(M)

View File

@ -1128,7 +1128,7 @@ protected:
Params params;
AggregatedDataVariants::Type method;
AggregatedDataVariants::Type method_chosen;
Sizes key_sizes;
AggregateFunctionsPlainPtrs aggregate_functions;
@ -1215,7 +1215,6 @@ protected:
size_t rows,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions,
const Sizes & key_sizes,
StringRefs & keys,
bool no_more_keys,
AggregateDataPtr overflow_row) const;
@ -1229,7 +1228,6 @@ protected:
size_t rows,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions,
const Sizes & key_sizes,
StringRefs & keys,
AggregateDataPtr overflow_row) const;
@ -1256,7 +1254,6 @@ public:
size_t rows,
ColumnRawPtrs & key_columns,
AggregateColumns & aggregate_columns,
const Sizes & key_sizes,
StringRefs & keys,
bool no_more_keys,
AggregateDataPtr overflow_row) const;
@ -1269,7 +1266,6 @@ public:
size_t rows,
ColumnRawPtrs & key_columns,
AggregateColumns & aggregate_columns,
const Sizes & key_sizes,
StringRefs & keys,
AggregateDataPtr overflow_row) const;
@ -1317,7 +1313,6 @@ protected:
MutableColumns & key_columns,
AggregateColumnsData & aggregate_columns,
MutableColumns & final_aggregate_columns,
const Sizes & key_sizes,
bool final) const;
template <typename Method, typename Table>
@ -1325,16 +1320,14 @@ protected:
Method & method,
Table & data,
MutableColumns & key_columns,
MutableColumns & final_aggregate_columns,
const Sizes & key_sizes) const;
MutableColumns & final_aggregate_columns) const;
template <typename Method, typename Table>
void convertToBlockImplNotFinal(
Method & method,
Table & data,
MutableColumns & key_columns,
AggregateColumnsData & aggregate_columns,
const Sizes & key_sizes) const;
AggregateColumnsData & aggregate_columns) const;
template <typename Filler>
Block prepareBlockAndFill(
@ -1364,7 +1357,6 @@ protected:
template <bool no_more_keys, typename Method, typename Table>
void mergeStreamsImplCase(
Block & block,
const Sizes & key_sizes,
Arena * aggregates_pool,
Method & method,
Table & data,
@ -1373,7 +1365,6 @@ protected:
template <typename Method, typename Table>
void mergeStreamsImpl(
Block & block,
const Sizes & key_sizes,
Arena * aggregates_pool,
Method & method,
Table & data,
@ -1393,7 +1384,6 @@ protected:
Method & method,
Arena * pool,
ColumnRawPtrs & key_columns,
const Sizes & key_sizes,
StringRefs & keys,
const Block & source,
std::vector<Block> & destinations) const;

View File

@ -186,8 +186,6 @@ void ExpressionAction::prepare(Block & sample_block)
all_const = false;
}
ColumnPtr new_column;
/// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function.
if (all_const && function->isSuitableForConstantFolding())
{
@ -635,7 +633,7 @@ void ExpressionActions::prependProjectInput()
actions.insert(actions.begin(), ExpressionAction::project(getRequiredColumns()));
}
void ExpressionActions::prependArrayJoin(const ExpressionAction & action, const Block & sample_block)
void ExpressionActions::prependArrayJoin(const ExpressionAction & action, const Block & sample_block_before)
{
if (action.type != ExpressionAction::ARRAY_JOIN)
throw Exception("ARRAY_JOIN action expected", ErrorCodes::LOGICAL_ERROR);
@ -651,7 +649,7 @@ void ExpressionActions::prependArrayJoin(const ExpressionAction & action, const
}
for (const std::string & name : array_join_set)
{
input_columns.emplace_back(name, sample_block.getByName(name).type);
input_columns.emplace_back(name, sample_block_before.getByName(name).type);
actions.insert(actions.begin(), ExpressionAction::removeColumn(name));
}

View File

@ -166,7 +166,7 @@ public:
/// Add the specified ARRAY JOIN action to the beginning. Change the appropriate input types to arrays.
/// If there are unknown columns in the ARRAY JOIN list, take their types from sample_block, and immediately after ARRAY JOIN remove them.
void prependArrayJoin(const ExpressionAction & action, const Block & sample_block);
void prependArrayJoin(const ExpressionAction & action, const Block & sample_block_before);
/// If the last action is ARRAY JOIN, and it does not affect the columns from required_columns, discard and return it.
/// Change the corresponding output types to arrays.

View File

@ -167,7 +167,7 @@ void removeDuplicateColumns(NamesAndTypesList & columns)
}
ExpressionAnalyzer::ExpressionAnalyzer(
const ASTPtr & ast_,
const ASTPtr & query_,
const Context & context_,
const StoragePtr & storage_,
const NamesAndTypesList & source_columns_,
@ -175,13 +175,13 @@ ExpressionAnalyzer::ExpressionAnalyzer(
size_t subquery_depth_,
bool do_global_,
const SubqueriesForSets & subqueries_for_set_)
: ast(ast_), context(context_), settings(context.getSettings()),
: query(query_), context(context_), settings(context.getSettings()),
subquery_depth(subquery_depth_),
source_columns(source_columns_), required_result_columns(required_result_columns_),
storage(storage_),
do_global(do_global_), subqueries_for_sets(subqueries_for_set_)
{
select_query = typeid_cast<ASTSelectQuery *>(ast.get());
select_query = typeid_cast<ASTSelectQuery *>(query.get());
if (!storage && select_query)
{
@ -217,7 +217,7 @@ ExpressionAnalyzer::ExpressionAnalyzer(
LogicalExpressionsOptimizer(select_query, settings).perform();
/// Creates a dictionary `aliases`: alias -> ASTPtr
addASTAliases(ast);
addASTAliases(query);
/// Common subexpression elimination. Rewrite rules.
normalizeTree();
@ -290,7 +290,7 @@ void ExpressionAnalyzer::translateQualifiedNames()
tables.emplace_back(getTableNameWithAliasFromTableExpression(join_table_expression, context));
}
translateQualifiedNamesImpl(ast, tables);
translateQualifiedNamesImpl(query, tables);
}
void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const std::vector<DatabaseAndTableWithAlias> & tables)
@ -384,7 +384,7 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const std::ve
void ExpressionAnalyzer::optimizeIfWithConstantCondition()
{
optimizeIfWithConstantConditionImpl(ast, aliases);
optimizeIfWithConstantConditionImpl(query);
}
bool ExpressionAnalyzer::tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const
@ -421,7 +421,7 @@ bool ExpressionAnalyzer::tryExtractConstValueFromCondition(const ASTPtr & condit
return false;
}
void ExpressionAnalyzer::optimizeIfWithConstantConditionImpl(ASTPtr & current_ast, ExpressionAnalyzer::Aliases & aliases) const
void ExpressionAnalyzer::optimizeIfWithConstantConditionImpl(ASTPtr & current_ast)
{
if (!current_ast)
return;
@ -431,11 +431,11 @@ void ExpressionAnalyzer::optimizeIfWithConstantConditionImpl(ASTPtr & current_as
ASTFunction * function_node = typeid_cast<ASTFunction *>(child.get());
if (!function_node || function_node->name != "if")
{
optimizeIfWithConstantConditionImpl(child, aliases);
optimizeIfWithConstantConditionImpl(child);
continue;
}
optimizeIfWithConstantConditionImpl(function_node->arguments, aliases);
optimizeIfWithConstantConditionImpl(function_node->arguments);
ASTExpressionList * args = typeid_cast<ASTExpressionList *>(function_node->arguments.get());
if (args->children.size() != 3)
@ -516,7 +516,7 @@ void ExpressionAnalyzer::analyzeAggregation()
}
}
getAggregates(ast, temp_actions);
getAggregates(query, temp_actions);
if (has_aggregation)
{
@ -592,10 +592,10 @@ void ExpressionAnalyzer::analyzeAggregation()
void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables()
{
/// Adds existing external tables (not subqueries) to the external_tables dictionary.
findExternalTables(ast);
findExternalTables(query);
/// Converts GLOBAL subqueries to external tables; Puts them into the external_tables dictionary: name -> StoragePtr.
initGlobalSubqueries(ast);
initGlobalSubqueries(query);
}
@ -609,18 +609,18 @@ void ExpressionAnalyzer::initGlobalSubqueries(ASTPtr & ast)
/// Bottom-up actions.
if (ASTFunction * node = typeid_cast<ASTFunction *>(ast.get()))
if (ASTFunction * func = typeid_cast<ASTFunction *>(ast.get()))
{
/// For GLOBAL IN.
if (do_global && (node->name == "globalIn" || node->name == "globalNotIn"))
addExternalStorage(node->arguments->children.at(1));
if (do_global && (func->name == "globalIn" || func->name == "globalNotIn"))
addExternalStorage(func->arguments->children.at(1));
}
else if (ASTTablesInSelectQueryElement * node = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
else if (ASTTablesInSelectQueryElement * table_elem = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
{
/// For GLOBAL JOIN.
if (do_global && node->table_join
&& static_cast<const ASTTableJoin &>(*node->table_join).locality == ASTTableJoin::Locality::Global)
addExternalStorage(node->table_expression);
if (do_global && table_elem->table_join
&& static_cast<const ASTTableJoin &>(*table_elem->table_join).locality == ASTTableJoin::Locality::Global)
addExternalStorage(table_elem->table_expression);
}
}
@ -916,11 +916,11 @@ void ExpressionAnalyzer::normalizeTree()
{
SetOfASTs tmp_set;
MapOfASTs tmp_map;
normalizeTreeImpl(ast, tmp_map, tmp_set, "", 0);
normalizeTreeImpl(query, tmp_map, tmp_set, "", 0);
try
{
ast->checkSize(settings.max_expanded_ast_elements);
query->checkSize(settings.max_expanded_ast_elements);
}
catch (Exception & e)
{
@ -1022,10 +1022,10 @@ void ExpressionAnalyzer::normalizeTreeImpl(
}
}
}
else if (ASTExpressionList * node = typeid_cast<ASTExpressionList *>(ast.get()))
else if (ASTExpressionList * expr_list = typeid_cast<ASTExpressionList *>(ast.get()))
{
/// Replace * with a list of columns.
ASTs & asts = node->children;
ASTs & asts = expr_list->children;
for (int i = static_cast<int>(asts.size()) - 1; i >= 0; --i)
{
if (typeid_cast<ASTAsterisk *>(asts[i].get()))
@ -1047,11 +1047,11 @@ void ExpressionAnalyzer::normalizeTreeImpl(
}
}
}
else if (ASTTablesInSelectQueryElement * node = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
else if (ASTTablesInSelectQueryElement * tables_elem = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
{
if (node->table_expression)
if (tables_elem->table_expression)
{
auto & database_and_table_name = static_cast<ASTTableExpression &>(*node->table_expression).database_and_table_name;
auto & database_and_table_name = static_cast<ASTTableExpression &>(*tables_elem->table_expression).database_and_table_name;
if (database_and_table_name)
{
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(database_and_table_name.get()))
@ -1130,18 +1130,18 @@ void ExpressionAnalyzer::addAliasColumns()
if (!storage)
return;
const auto & aliases = storage->getColumns().aliases;
source_columns.insert(std::end(source_columns), std::begin(aliases), std::end(aliases));
const auto & storage_aliases = storage->getColumns().aliases;
source_columns.insert(std::end(source_columns), std::begin(storage_aliases), std::end(storage_aliases));
}
void ExpressionAnalyzer::executeScalarSubqueries()
{
if (!select_query)
executeScalarSubqueriesImpl(ast);
executeScalarSubqueriesImpl(query);
else
{
for (auto & child : ast->children)
for (auto & child : query->children)
{
/// Do not go to FROM, JOIN, UNION.
if (!typeid_cast<const ASTTableExpression *>(child.get())
@ -1198,8 +1198,8 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
ASTPtr query = subquery->children.at(0);
BlockIO res = InterpreterSelectWithUnionQuery(query, subquery_context, {}, QueryProcessingStage::Complete, subquery_depth + 1).execute();
ASTPtr subquery_select = subquery->children.at(0);
BlockIO res = InterpreterSelectWithUnionQuery(subquery_select, subquery_context, {}, QueryProcessingStage::Complete, subquery_depth + 1).execute();
Block block;
try
@ -1662,9 +1662,9 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block &
const ASTPtr & left_arg = args.children.at(0);
const ASTPtr & right_arg = args.children.at(1);
auto getTupleTypeFromAst = [this](const ASTPtr & node) -> DataTypePtr
auto getTupleTypeFromAst = [this](const ASTPtr & tuple_ast) -> DataTypePtr
{
auto ast_function = typeid_cast<const ASTFunction *>(node.get());
auto ast_function = typeid_cast<const ASTFunction *>(tuple_ast.get());
if (ast_function && ast_function->name == "tuple" && !ast_function->arguments->children.empty())
{
/// Won't parse all values of outer tuple.
@ -1673,7 +1673,7 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block &
return std::make_shared<DataTypeTuple>(DataTypes({value_raw.second}));
}
return evaluateConstantExpression(node, context).second;
return evaluateConstantExpression(tuple_ast, context).second;
};
const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type;
@ -1855,7 +1855,7 @@ void ExpressionAnalyzer::getArrayJoinedColumns()
array_join_name_to_alias[nested_table_name] = nested_table_alias;
}
getArrayJoinedColumnsImpl(ast);
getArrayJoinedColumnsImpl(query);
/// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column,
/// to get the correct number of rows.
@ -1996,7 +1996,7 @@ void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join,
ScopeStack scopes(actions, settings);
ProjectionManipulatorPtr projection_manipulator;
if (!isThereArrayJoin(ast) && settings.enable_conditional_computation && !only_consts)
if (!isThereArrayJoin(query) && settings.enable_conditional_computation && !only_consts)
projection_manipulator = std::make_shared<ConditionalTree>(scopes, context);
else
projection_manipulator = std::make_shared<DefaultProjectionManipulator>(scopes);
@ -2291,12 +2291,12 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
}
}
}
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(ast.get()))
else if (ASTLiteral * literal = typeid_cast<ASTLiteral *>(ast.get()))
{
DataTypePtr type = applyVisitor(FieldToDataType(), node->value);
DataTypePtr type = applyVisitor(FieldToDataType(), literal->value);
ColumnWithTypeAndName column;
column.column = type->createColumnConst(1, convertFieldToType(node->value, *type));
column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type));
column.type = type;
column.name = getColumnName();
@ -2457,13 +2457,13 @@ void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only
}
void ExpressionAnalyzer::AnalyzedJoin::createJoinedBlockActions(const ASTSelectQuery * select_query,
void ExpressionAnalyzer::AnalyzedJoin::createJoinedBlockActions(const ASTSelectQuery * select_query_with_join,
const Context & context)
{
if (!select_query)
if (!select_query_with_join)
return;
const ASTTablesInSelectQueryElement * join = select_query->join();
const ASTTablesInSelectQueryElement * join = select_query_with_join->join();
if (!join)
return;
@ -2487,7 +2487,7 @@ void ExpressionAnalyzer::AnalyzedJoin::createJoinedBlockActions(const ASTSelectQ
required_columns_from_joined_table.insert(required_columns_from_joined_table.end(),
required_columns_set.begin(), required_columns_set.end());
const auto & source_columns_name = getColumnsFromJoinedTable(context, select_query);
const auto & source_columns_name = getColumnsFromJoinedTable(context, select_query_with_join);
ExpressionAnalyzer analyzer(expression_list, context, nullptr, source_columns_name, required_columns_from_joined_table);
joined_block_actions = analyzer.getActions(false);
@ -2506,11 +2506,11 @@ NamesAndTypesList ExpressionAnalyzer::AnalyzedJoin::getColumnsAddedByJoin() cons
return result;
}
NamesAndTypesList ExpressionAnalyzer::AnalyzedJoin::getColumnsFromJoinedTable(const Context & context, const ASTSelectQuery * select_query)
NamesAndTypesList ExpressionAnalyzer::AnalyzedJoin::getColumnsFromJoinedTable(const Context & context, const ASTSelectQuery * select_query_with_join)
{
if (select_query && !columns_from_joined_table.size())
if (select_query_with_join && !columns_from_joined_table.size())
{
if (const ASTTablesInSelectQueryElement * node = select_query->join())
if (const ASTTablesInSelectQueryElement * node = select_query_with_join->join())
{
Block nested_result_sample;
const auto & table_expression = static_cast<const ASTTableExpression &>(*node->table_expression);
@ -2902,10 +2902,10 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result)
ASTs asts;
if (auto node = typeid_cast<const ASTExpressionList *>(ast.get()))
if (auto node = typeid_cast<const ASTExpressionList *>(query.get()))
asts = node->children;
else
asts = ASTs(1, ast);
asts = ASTs(1, query);
for (size_t i = 0; i < asts.size(); ++i)
{
@ -2941,7 +2941,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions()
{
ExpressionActionsPtr actions = std::make_shared<ExpressionActions>(NamesAndTypesList(), settings);
getRootActions(ast, true, true, actions);
getRootActions(query, true, true, actions);
return actions;
}
@ -3001,7 +3001,7 @@ void ExpressionAnalyzer::collectUsedColumns()
for (const auto & left_key_ast : analyzed_join.key_asts_left)
getRequiredSourceColumnsImpl(left_key_ast, available_columns, required, ignored, {}, required_joined_columns);
getRequiredSourceColumnsImpl(ast, available_columns, required, ignored, available_joined_columns, required_joined_columns);
getRequiredSourceColumnsImpl(query, available_columns, required, ignored, available_joined_columns, required_joined_columns);
for (auto it = analyzed_join.columns_added_by_join.begin(); it != analyzed_join.columns_added_by_join.end();)
{

View File

@ -103,7 +103,7 @@ private:
public:
ExpressionAnalyzer(
const ASTPtr & ast_,
const ASTPtr & query_,
const Context & context_,
const StoragePtr & storage_,
const NamesAndTypesList & source_columns_ = {},
@ -182,7 +182,7 @@ public:
bool isRewriteSubqueriesPredicate() { return rewrite_subqueries; }
private:
ASTPtr ast;
ASTPtr query;
ASTSelectQuery * select_query;
const Context & context;
Settings settings;
@ -261,11 +261,11 @@ private:
/// Actions which need to be calculated on joined block.
ExpressionActionsPtr joined_block_actions;
void createJoinedBlockActions(const ASTSelectQuery * select_query, const Context & context);
void createJoinedBlockActions(const ASTSelectQuery * select_query_with_join, const Context & context);
NamesAndTypesList getColumnsAddedByJoin() const;
NamesAndTypesList getColumnsFromJoinedTable(const Context & context, const ASTSelectQuery * select_query);
NamesAndTypesList getColumnsFromJoinedTable(const Context & context, const ASTSelectQuery * select_query_with_join);
};
AnalyzedJoin analyzed_join;
@ -330,7 +330,7 @@ private:
/// remove Function_if AST if condition is constant
void optimizeIfWithConstantCondition();
void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast, Aliases & aliases) const;
void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast);
bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const;
void makeSet(const ASTFunction * node, const Block & sample_block);

View File

@ -260,7 +260,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
const auto last_modified = config_repository->getLastModificationTime(config_path);
if (force_reload || last_modified > config_last_modified)
{
auto config = config_repository->load(config_path);
auto loaded_config = config_repository->load(config_path);
loadable_objects_defined_in_config[config_path].clear();
@ -272,7 +272,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
/// get all objects' definitions
Poco::Util::AbstractConfiguration::Keys keys;
config->keys(keys);
loaded_config->keys(keys);
/// for each loadable object defined in xml config
for (const auto & key : keys)
@ -289,7 +289,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
try
{
name = config->getString(key + "." + config_settings.external_name);
name = loaded_config->getString(key + "." + config_settings.external_name);
if (name.empty())
{
LOG_WARNING(log, config_path << ": " + config_settings.external_name + " name cannot be empty");
@ -312,7 +312,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
+ " already declared in file " + object_it->second.origin,
ErrorCodes::EXTERNAL_LOADABLE_ALREADY_EXISTS);
auto object_ptr = create(name, *config, key);
auto object_ptr = create(name, *loaded_config, key);
/// If the object could not be loaded.
if (const auto exception_ptr = object_ptr->getCreationException())

View File

@ -148,9 +148,11 @@ void InJoinSubqueriesPreprocessor::process(ASTSelectQuery * query) const
return;
/// If not really distributed table, skip it.
StoragePtr storage = tryGetTable(table_expression->database_and_table_name, context);
if (!storage || !hasAtLeastTwoShards(*storage))
return;
{
StoragePtr storage = tryGetTable(table_expression->database_and_table_name, context);
if (!storage || !hasAtLeastTwoShards(*storage))
return;
}
forEachNonGlobalSubquery(query, [&] (IAST * subquery, IAST * function, IAST * table_join)
{

View File

@ -392,7 +392,7 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression
}
void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputStreamPtr & input, bool dry_run)
void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run)
{
/** Streams of data. When the query is executed in parallel, we have several data streams.
* If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then
@ -429,8 +429,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
if (const StorageMergeTree * merge_tree = dynamic_cast<const StorageMergeTree *>(storage.get()))
optimize_prewhere(*merge_tree);
else if (const StorageReplicatedMergeTree * merge_tree = dynamic_cast<const StorageReplicatedMergeTree *>(storage.get()))
optimize_prewhere(*merge_tree);
else if (const StorageReplicatedMergeTree * replicated_merge_tree = dynamic_cast<const StorageReplicatedMergeTree *>(storage.get()))
optimize_prewhere(*replicated_merge_tree);
}
AnalysisResult expressions;
@ -448,8 +448,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
}
else
{
if (input)
pipeline.streams.push_back(input);
if (prepared_input)
pipeline.streams.push_back(prepared_input);
expressions = analyzeExpressions(from_stage, false);

View File

@ -126,7 +126,7 @@ private:
}
};
void executeImpl(Pipeline & pipeline, const BlockInputStreamPtr & input, bool dry_run);
void executeImpl(Pipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run);
struct AnalysisResult

View File

@ -216,15 +216,15 @@ BlockIO InterpreterSystemQuery::execute()
}
StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_name, const String & table_name, Context & context)
StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_name, const String & table_name, Context & system_context)
{
auto database = context.getDatabase(database_name);
auto table_ddl_guard = context.getDDLGuard(database_name, table_name, "Table " + database_name + "." + table_name + " is restarting right now");
auto database = system_context.getDatabase(database_name);
auto table_ddl_guard = system_context.getDDLGuard(database_name, table_name, "Table " + database_name + "." + table_name + " is restarting right now");
ASTPtr create_ast;
/// Detach actions
{
auto table = context.tryGetTable(database_name, table_name);
auto table = system_context.tryGetTable(database_name, table_name);
if (!table || !dynamic_cast<const StorageReplicatedMergeTree *>(table.get()))
return nullptr;
@ -233,7 +233,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__);
create_ast = context.getCreateTableQuery(database_name, table_name);
create_ast = system_context.getCreateTableQuery(database_name, table_name);
database->detachTable(table_name);
}
@ -245,35 +245,35 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam
create.attach = true;
std::string data_path = database->getDataPath();
auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns, context);
auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns, system_context);
StoragePtr table = StorageFactory::instance().get(create,
data_path,
table_name,
database_name,
context,
context.getGlobalContext(),
system_context,
system_context.getGlobalContext(),
columns,
create.attach,
false);
database->createTable(context, table_name, table, create_ast);
database->createTable(system_context, table_name, table, create_ast);
table->startup();
return table;
}
}
void InterpreterSystemQuery::restartReplicas(Context & context)
void InterpreterSystemQuery::restartReplicas(Context & system_context)
{
std::vector<std::pair<String, String>> replica_names;
for (auto & elem : context.getDatabases())
for (auto & elem : system_context.getDatabases())
{
DatabasePtr & database = elem.second;
const String & database_name = elem.first;
for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next())
for (auto iterator = database->getIterator(system_context); iterator->isValid(); iterator->next())
{
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
replica_names.emplace_back(database_name, iterator->name());
@ -285,7 +285,7 @@ void InterpreterSystemQuery::restartReplicas(Context & context)
ThreadPool pool(std::min(getNumberOfPhysicalCPUCores(), replica_names.size()));
for (auto & table : replica_names)
pool.schedule([&] () { tryRestartReplica(table.first, table.second, context); });
pool.schedule([&] () { tryRestartReplica(table.first, table.second, system_context); });
pool.wait();
}

View File

@ -29,7 +29,7 @@ private:
/// Returns pointer to a newly created table if the restart was successful
StoragePtr tryRestartReplica(const String & database_name, const String & table_name, Context & context);
void restartReplicas(Context & context);
void restartReplicas(Context & system_context);
void syncReplica(ASTSystemQuery & query);
};

View File

@ -231,11 +231,11 @@ ProcessListEntry::~ProcessListEntry()
auto range = user_process_list.queries.equal_range(query_id);
if (range.first != range.second)
{
for (auto it = range.first; it != range.second; ++it)
for (auto jt = range.first; jt != range.second; ++jt)
{
if (it->second == process_list_element_ptr)
if (jt->second == process_list_element_ptr)
{
user_process_list.queries.erase(it);
user_process_list.queries.erase(jt);
found = true;
break;
}

View File

@ -103,7 +103,6 @@ void NO_INLINE Aggregator::executeSpecialized(
size_t rows,
ColumnRawPtrs & key_columns,
AggregateColumns & aggregate_columns,
const Sizes & key_sizes,
StringRefs & keys,
bool no_more_keys,
AggregateDataPtr overflow_row) const
@ -113,10 +112,10 @@ void NO_INLINE Aggregator::executeSpecialized(
if (!no_more_keys)
executeSpecializedCase<false, Method, AggregateFunctionsList>(
method, state, aggregates_pool, rows, key_columns, aggregate_columns, key_sizes, keys, overflow_row);
method, state, aggregates_pool, rows, key_columns, aggregate_columns, keys, overflow_row);
else
executeSpecializedCase<true, Method, AggregateFunctionsList>(
method, state, aggregates_pool, rows, key_columns, aggregate_columns, key_sizes, keys, overflow_row);
method, state, aggregates_pool, rows, key_columns, aggregate_columns, keys, overflow_row);
}
#pragma GCC diagnostic push
@ -130,7 +129,6 @@ void NO_INLINE Aggregator::executeSpecializedCase(
size_t rows,
ColumnRawPtrs & key_columns,
AggregateColumns & aggregate_columns,
const Sizes & key_sizes,
StringRefs & keys,
AggregateDataPtr overflow_row) const
{
@ -143,7 +141,7 @@ void NO_INLINE Aggregator::executeSpecializedCase(
bool overflow = false; /// New key did not fit in the hash table because of no_more_keys.
/// Get the key to insert into the hash table.
typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
typename Method::Key key = state.getKey(key_columns, params.keys_size, i, keys, *aggregates_pool);
if (!no_more_keys) /// Insert.
{

View File

@ -269,7 +269,6 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
res.finish_callback = [elem, &context, log_queries] (IBlockInputStream * stream_in, IBlockOutputStream * stream_out) mutable
{
QueryStatus * process_list_elem = context.getProcessListElement();
const Settings & settings = context.getSettingsRef();
if (!process_list_elem)
return;
@ -277,7 +276,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Update performance counters before logging to query_log
CurrentThread::finalizePerformanceCounters();
QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events);
QueryStatusInfo info = process_list_elem->getInfo(true, context.getSettingsRef().log_profile_events);
double elapsed_seconds = info.elapsed_seconds;
@ -345,14 +344,14 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.exception = getCurrentExceptionMessage(false);
QueryStatus * process_list_elem = context.getProcessListElement();
const Settings & settings = context.getSettingsRef();
const Settings & current_settings = context.getSettingsRef();
/// Update performance counters before logging to query_log
CurrentThread::finalizePerformanceCounters();
if (process_list_elem)
{
QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events, false);
QueryStatusInfo info = process_list_elem->getInfo(true, current_settings.log_profile_events, false);
elem.query_duration_ms = info.elapsed_seconds * 1000;
@ -365,7 +364,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.profile_counters = std::move(info.profile_counters);
}
if (settings.calculate_text_stack_trace)
if (current_settings.calculate_text_stack_trace)
setExceptionStackTrace(elem);
logException(context, elem);

View File

@ -136,7 +136,7 @@ void sortBlock(Block & block, const SortDescription & description, size_t limit)
bool need_collation = false;
ColumnsWithSortDescriptions columns_with_sort_desc = getColumnsWithSortDescription(block, description);
for (size_t i = 0, size = description.size(); i < size; ++i)
for (size_t i = 0, num_sort_columns = description.size(); i < num_sort_columns; ++i)
{
if (needCollation(columns_with_sort_desc[i].first, description[i]))
{

View File

@ -69,7 +69,7 @@ namespace ZeroTraits
template <>
inline void set<CompactStringRef>(CompactStringRef & x) { x.data_mixed = nullptr; }
};
}
template <>
struct DefaultHash<CompactStringRef>

View File

@ -58,7 +58,7 @@ namespace ZeroTraits \
\
template <> \
inline void set<STRUCT>(STRUCT & x) { x.size = 0; } \
}; \
} \
\
template <> \
struct DefaultHash<STRUCT> \

View File

@ -61,7 +61,7 @@ namespace ZeroTraits \
\
template <> \
inline void set<STRUCT>(STRUCT & x) { x.data = nullptr; } \
}; \
} \
\
template <> \
struct DefaultHash<STRUCT> \

View File

@ -79,7 +79,7 @@ namespace ZeroTraits
template <>
inline void set<SmallStringRef>(SmallStringRef & x) { x.size = 0; }
};
}
template <>
struct DefaultHash<SmallStringRef>

View File

@ -8,12 +8,12 @@ namespace DB
void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
auto format_element = [&](const String & name)
auto format_element = [&](const String & elem_name)
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
WriteBufferFromOStream wb(settings.ostr, 32);
settings.writeIdentifier(name, wb);
settings.writeIdentifier(elem_name, wb);
wb.next();
settings.ostr << (settings.hilite ? hilite_none : "");

View File

@ -410,5 +410,5 @@ void ASTSelectQuery::addTableFunction(ASTPtr & table_function_ptr)
table_expression->database_and_table_name = nullptr;
}
};
}

View File

@ -233,8 +233,8 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
&& contents_begin[8] >= '0' && contents_begin[8] <= '9'
&& contents_begin[9] >= '0' && contents_begin[9] <= '9')
{
std::string contents(contents_begin, contents_end - contents_begin);
throw Exception("Argument of function toDate is unquoted: toDate(" + contents + "), must be: toDate('" + contents + "')"
std::string contents_str(contents_begin, contents_end - contents_begin);
throw Exception("Argument of function toDate is unquoted: toDate(" + contents_str + "), must be: toDate('" + contents_str + "')"
, ErrorCodes::SYNTAX_ERROR);
}
@ -508,7 +508,7 @@ bool ParserStringLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
{
readQuotedStringWithSQLStyle(s, in);
}
catch (const Exception & e)
catch (const Exception &)
{
expected.add(pos, "string literal");
return false;

View File

@ -75,14 +75,14 @@ BlockInputStreams StorageSystemReplicas::read(
}
/// Do you need columns that require a walkthrough in ZooKeeper to compute.
/// Do you need columns that require a ZooKeeper request to compute.
bool with_zk_fields = false;
for (const auto & name : column_names)
for (const auto & column_name : column_names)
{
if ( name == "log_max_index"
|| name == "log_pointer"
|| name == "total_replicas"
|| name == "active_replicas")
if ( column_name == "log_max_index"
|| column_name == "log_pointer"
|| column_name == "total_replicas"
|| column_name == "active_replicas")
{
with_zk_fields = true;
break;

View File

@ -1,9 +1,11 @@
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <gtest/gtest.h>
#pragma GCC diagnostic pop
#include <DataStreams/ColumnGathererStream.h>
using DB::RowSourcePart;
static void check(const RowSourcePart & s, size_t num, bool flag)

View File

@ -57,7 +57,7 @@ public:
static constexpr char DEFAULT_GRAPHITE_CONFIG_NAME[] = "graphite";
BaseDaemon();
~BaseDaemon();
~BaseDaemon() override;
/// Загружает конфигурацию и "строит" логгеры на запись в файлы
void initialize(Poco::Util::Application &) override;

View File

@ -39,7 +39,7 @@ private:
void writeImpl(const T & data, time_t timestamp, const std::string & custom_root_path)
{
if (!timestamp)
timestamp = time(0);
timestamp = time(nullptr);
try
{

View File

@ -14,18 +14,13 @@ namespace DB
class OwnSplitChannel : public Poco::Channel
{
public:
OwnSplitChannel() = default;
/// Makes an extended message from msg and passes it to the client logs queue and child (if possible)
void log(const Poco::Message & msg) override;
/// Adds a child channel
void addChannel(Poco::AutoPtr<Poco::Channel> channel);
~OwnSplitChannel() = default;
private:
using ChannelPtr = Poco::AutoPtr<Poco::Channel>;
/// Handler and its pointer casted to extended interface
using ExtendedChannelPtrPair = std::pair<ChannelPtr, ExtendedLogChannel *>;