Resolve conflict

This commit is contained in:
alesapin 2018-09-03 13:11:10 +03:00
commit 11d248ba10
194 changed files with 2052 additions and 1494 deletions

View File

@ -4,27 +4,27 @@ matrix:
fast_finish: true
include:
# We need to have gcc7 headers to compile c++17 code on clang
- os: linux
cache:
ccache: true
timeout: 1000
directories:
- /home/travis/.ccache
addons:
apt:
update: true
sources:
- ubuntu-toolchain-r-test
- llvm-toolchain-trusty-5.0
packages: [ ninja-build, g++-7, clang-5.0, lld-5.0, libicu-dev, libreadline-dev, libmysqlclient-dev, unixodbc-dev, libltdl-dev, libssl-dev, libboost-dev, zlib1g-dev, libdouble-conversion-dev, libsparsehash-dev, librdkafka-dev, libcapnp-dev, libsparsehash-dev, libgoogle-perftools-dev, bash, expect, python, python-lxml, python-termcolor, curl, perl, sudo, openssl]
env:
- MATRIX_EVAL="export CC=clang-5.0 CXX=clang++-5.0"
script:
- utils/travis/normal.sh
# - os: linux
#
# cache:
# ccache: true
# timeout: 1000
# directories:
# - /home/travis/.ccache
#
# addons:
# apt:
# update: true
# sources:
# - ubuntu-toolchain-r-test
# - llvm-toolchain-trusty-5.0
# packages: [ ninja-build, g++-7, clang-5.0, lld-5.0, libicu-dev, libreadline-dev, libmysqlclient-dev, unixodbc-dev, libltdl-dev, libssl-dev, libboost-dev, zlib1g-dev, libdouble-conversion-dev, libsparsehash-dev, librdkafka-dev, libcapnp-dev, libsparsehash-dev, libgoogle-perftools-dev, bash, expect, python, python-lxml, python-termcolor, curl, perl, sudo, openssl]
#
# env:
# - MATRIX_EVAL="export CC=clang-5.0 CXX=clang++-5.0"
#
# script:
# - utils/travis/normal.sh
- os: linux

View File

@ -2,7 +2,7 @@
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-stringop-overflow")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-implicit-fallthrough -std=c++1z")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-implicit-fallthrough -Wno-class-memaccess -std=c++1z")
elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-format -Wno-parentheses-equality")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-format -std=c++1z")
@ -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 (NOT CMAKE_CXX_COMPILER_VERSION VERSION_LESS 6)
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

@ -403,6 +403,10 @@ public:
}
#ifndef __clang__
#pragma GCC optimize("-fno-var-tracking-assignments")
#endif
int mainEntryClickHouseBenchmark(int argc, char ** argv)
{
using namespace DB;

View File

@ -63,6 +63,10 @@
#include "Suggest.h"
#endif
#ifndef __clang__
#pragma GCC optimize("-fno-var-tracking-assignments")
#endif
/// http://en.wikipedia.org/wiki/ANSI_escape_code
@ -553,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

@ -108,7 +108,7 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
select->format(settings);
std::string query = ss.str();
if (Poco::Data::ODBC::Utility::isError(Poco::Data::ODBC::SQLPrepare(hstmt, reinterpret_cast<SQLCHAR *>(&query[0]), query.size())))
if (Poco::Data::ODBC::Utility::isError(Poco::Data::ODBC::SQLPrepare(hstmt, reinterpret_cast<SQLCHAR *>(query.data()), query.size())))
throw Poco::Data::ODBC::DescriptorException(session.dbc());
if (Poco::Data::ODBC::Utility::isError(SQLExecute(hstmt)))
@ -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

@ -31,6 +31,11 @@
#include <Poco/XML/XMLStream.h>
#include <Common/InterruptListener.h>
#ifndef __clang__
#pragma GCC optimize("-fno-var-tracking-assignments")
#endif
/** Tests launcher for ClickHouse.
* The tool walks through given or default folder in order to find files with
* tests' descriptions and launches it.
@ -1387,6 +1392,7 @@ static void getFilesFromDir(const fs::path & dir, std::vector<String> & input_fi
}
}
int mainEntryClickHousePerformanceTest(int argc, char ** argv)
try
{

View File

@ -85,7 +85,7 @@ void MetricsTransmitter::transmit(std::vector<ProfileEvents::Count> & prev_count
const auto counter_increment = counter - prev_counters[i];
prev_counters[i] = counter;
std::string key{ProfileEvents::getDescription(static_cast<ProfileEvents::Event>(i))};
std::string key{ProfileEvents::getName(static_cast<ProfileEvents::Event>(i))};
key_vals.emplace_back(profile_events_path_prefix + key, counter_increment);
}
}
@ -96,7 +96,7 @@ void MetricsTransmitter::transmit(std::vector<ProfileEvents::Count> & prev_count
{
const auto value = CurrentMetrics::values[i].load(std::memory_order_relaxed);
std::string key{CurrentMetrics::getDescription(static_cast<CurrentMetrics::Metric>(i))};
std::string key{CurrentMetrics::getName(static_cast<CurrentMetrics::Metric>(i))};
key_vals.emplace_back(current_metrics_path_prefix + key, value);
}
}

View File

@ -369,9 +369,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
#if defined(__linux__)
if (!TaskStatsInfoGetter::checkPermissions())
{
LOG_INFO(log, "It looks like the process has no CAP_NET_ADMIN capability, some performance statistics will be disabled."
LOG_INFO(log, "It looks like the process has no CAP_NET_ADMIN capability, 'taskstats' performance statistics will be disabled."
" It could happen due to incorrect ClickHouse package installation."
" You could resolve the problem manually with 'sudo setcap cap_net_admin=+ep /usr/bin/clickhouse'");
" You could resolve the problem manually with 'sudo setcap cap_net_admin=+ep /usr/bin/clickhouse'."
" Note that it will not work on 'nosuid' mounted filesystems."
" It also doesn't work if you run clickhouse-server inside network namespace as it happens in some containers.");
}
#else
LOG_INFO(log, "TaskStats is not implemented for this OS. IO accounting will be disabled.");

View File

@ -59,10 +59,11 @@ private:
size_t old_size = state.dynamic_array_size;
if (old_size < new_size)
{
state.array_of_aggregate_datas = arena.realloc(
state.array_of_aggregate_datas = arena.alignedRealloc(
state.array_of_aggregate_datas,
old_size * nested_size_of_data,
new_size * nested_size_of_data);
new_size * nested_size_of_data,
nested_func->alignOfData());
size_t i = old_size;
char * nested_state = state.array_of_aggregate_datas + i * nested_size_of_data;

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

@ -91,7 +91,7 @@ public:
const auto & value = this->data(place).value;
size_t size = value.size();
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(&value[0]), size * sizeof(value[0]));
buf.write(reinterpret_cast<const char *>(value.data()), size * sizeof(value[0]));
}
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override
@ -108,7 +108,7 @@ public:
auto & value = this->data(place).value;
value.resize(size, arena);
buf.read(reinterpret_cast<char *>(&value[0]), size * sizeof(value[0]));
buf.read(reinterpret_cast<char *>(value.data()), size * sizeof(value[0]));
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
@ -171,7 +171,7 @@ struct GroupArrayListNodeBase
UInt64 size;
readVarUInt(size, buf);
Node * node = reinterpret_cast<Node *>(arena->alloc(sizeof(Node) + size));
Node * node = reinterpret_cast<Node *>(arena->alignedAlloc(sizeof(Node) + size, alignof(Node)));
node->size = size;
buf.read(node->data(), size);
return node;
@ -187,7 +187,7 @@ struct GroupArrayListNodeString : public GroupArrayListNodeBase<GroupArrayListNo
{
StringRef string = static_cast<const ColumnString &>(column).getDataAt(row_num);
Node * node = reinterpret_cast<Node *>(arena->alloc(sizeof(Node) + string.size));
Node * node = reinterpret_cast<Node *>(arena->alignedAlloc(sizeof(Node) + string.size, alignof(Node)));
node->next = nullptr;
node->size = string.size;
memcpy(node->data(), string.data, string.size);
@ -207,7 +207,7 @@ struct GroupArrayListNodeGeneral : public GroupArrayListNodeBase<GroupArrayListN
static Node * allocate(const IColumn & column, size_t row_num, Arena * arena)
{
const char * begin = arena->alloc(sizeof(Node));
const char * begin = arena->alignedAlloc(sizeof(Node), alignof(Node));
StringRef value = column.serializeValueIntoArena(row_num, *arena, begin);
Node * node = reinterpret_cast<Node *>(const_cast<char *>(begin));

View File

@ -111,7 +111,7 @@ public:
const auto & value = this->data(place).value;
size_t size = value.size();
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(&value[0]), size * sizeof(value[0]));
buf.write(reinterpret_cast<const char *>(value.data()), size * sizeof(value[0]));
}
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override
@ -125,7 +125,7 @@ public:
auto & value = this->data(place).value;
value.resize(size, arena);
buf.read(reinterpret_cast<char *>(&value[0]), size * sizeof(value[0]));
buf.read(reinterpret_cast<char *>(value.data()), size * sizeof(value[0]));
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override

View File

@ -31,20 +31,23 @@ class AggregateFunctionNullBase : public IAggregateFunctionHelper<Derived>
{
protected:
AggregateFunctionPtr nested_function;
size_t prefix_size;
/** In addition to data for nested aggregate function, we keep a flag
* indicating - was there at least one non-NULL value accumulated.
* In case of no not-NULL values, the function will return NULL.
*
* We use prefix_size bytes for flag to satisfy the alignment requirement of nested state.
*/
static AggregateDataPtr nestedPlace(AggregateDataPtr place) noexcept
AggregateDataPtr nestedPlace(AggregateDataPtr place) const noexcept
{
return place + (result_is_nullable ? 1 : 0);
return place + prefix_size;
}
static ConstAggregateDataPtr nestedPlace(ConstAggregateDataPtr place) noexcept
ConstAggregateDataPtr nestedPlace(ConstAggregateDataPtr place) const noexcept
{
return place + (result_is_nullable ? 1 : 0);
return place + prefix_size;
}
static void initFlag(AggregateDataPtr place) noexcept
@ -68,6 +71,10 @@ public:
AggregateFunctionNullBase(AggregateFunctionPtr nested_function_)
: nested_function{nested_function_}
{
if (result_is_nullable)
prefix_size = nested_function->alignOfData();
else
prefix_size = 0;
}
String getName() const override
@ -101,12 +108,12 @@ public:
size_t sizeOfData() const override
{
return 1 + nested_function->sizeOfData();
return prefix_size + nested_function->sizeOfData();
}
size_t alignOfData() const override
{
return 1; /// NOTE This works fine on x86_64 and ok on AArch64. Doesn't work under UBSan.
return nested_function->alignOfData();
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override

View File

@ -134,7 +134,7 @@ public:
size_t old_size = data_to.size();
data_to.resize(data_to.size() + size);
data.getManyFloat(&levels.levels[0], &levels.permutation[0], size, &data_to[old_size]);
data.getManyFloat(levels.levels.data(), levels.permutation.data(), size, &data_to[old_size]);
}
else
{
@ -142,7 +142,7 @@ public:
size_t old_size = data_to.size();
data_to.resize(data_to.size() + size);
data.getMany(&levels.levels[0], &levels.permutation[0], size, &data_to[old_size]);
data.getMany(levels.levels.data(), levels.permutation.data(), size, &data_to[old_size]);
}
}
else

View File

@ -82,7 +82,7 @@ struct __attribute__((__packed__)) AggregateFunctionUniqUpToData
/// Write values only if the state is not overflowed. Otherwise, they are not needed, and only the fact that the state is overflowed is important.
if (count <= threshold)
wb.write(reinterpret_cast<const char *>(&data[0]), count * sizeof(data[0]));
wb.write(reinterpret_cast<const char *>(data), count * sizeof(data[0]));
}
void read(ReadBuffer & rb, UInt8 threshold)
@ -90,7 +90,7 @@ struct __attribute__((__packed__)) AggregateFunctionUniqUpToData
readBinary(count, rb);
if (count <= threshold)
rb.read(reinterpret_cast<char *>(&data[0]), count * sizeof(data[0]));
rb.read(reinterpret_cast<char *>(data), count * sizeof(data[0]));
}
void add(const IColumn & column, size_t row_num, UInt8 threshold)

View File

@ -53,7 +53,7 @@ struct QuantileExact
{
size_t size = array.size();
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(&array[0]), size * sizeof(array[0]));
buf.write(reinterpret_cast<const char *>(array.data()), size * sizeof(array[0]));
}
void deserialize(ReadBuffer & buf)
@ -61,7 +61,7 @@ struct QuantileExact
size_t size = 0;
readVarUInt(size, buf);
array.resize(size);
buf.read(reinterpret_cast<char *>(&array[0]), size * sizeof(array[0]));
buf.read(reinterpret_cast<char *>(array.data()), size * sizeof(array[0]));
}
/// Get the value of the `level` quantile. The level must be between 0 and 1.

View File

@ -136,7 +136,7 @@ class QuantileTDigest
{
if (unmerged > 0)
{
RadixSort<RadixSortTraits>::execute(&summary[0], summary.size());
RadixSort<RadixSortTraits>::execute(summary.data(), summary.size());
if (summary.size() > 3)
{
@ -212,7 +212,7 @@ public:
{
compress();
writeVarUInt(summary.size(), buf);
buf.write(reinterpret_cast<const char *>(&summary[0]), summary.size() * sizeof(summary[0]));
buf.write(reinterpret_cast<const char *>(summary.data()), summary.size() * sizeof(summary[0]));
}
void deserialize(ReadBuffer & buf)
@ -224,7 +224,7 @@ public:
throw Exception("Too large t-digest summary size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
summary.resize(size);
buf.read(reinterpret_cast<char *>(&summary[0]), size * sizeof(summary[0]));
buf.read(reinterpret_cast<char *>(summary.data()), size * sizeof(summary[0]));
}
/** Calculates the quantile q [0, 1] based on the digest.

View File

@ -158,7 +158,7 @@ namespace detail
void serialize(WriteBuffer & buf) const
{
writeBinary(elems.size(), buf);
buf.write(reinterpret_cast<const char *>(&elems[0]), elems.size() * sizeof(elems[0]));
buf.write(reinterpret_cast<const char *>(elems.data()), elems.size() * sizeof(elems[0]));
}
void deserialize(ReadBuffer & buf)
@ -166,7 +166,7 @@ namespace detail
size_t size = 0;
readBinary(size, buf);
elems.resize(size);
buf.readStrict(reinterpret_cast<char *>(&elems[0]), size * sizeof(elems[0]));
buf.readStrict(reinterpret_cast<char *>(elems.data()), size * sizeof(elems[0]));
}
UInt16 get(double level) const

View File

@ -3,11 +3,15 @@
#include <DataStreams/ColumnGathererStream.h>
#include <IO/WriteBufferFromArena.h>
#include <Common/SipHash.h>
#include <Common/AlignedBuffer.h>
#include <Common/typeid_cast.h>
#include <Common/Arena.h>
#include <Columns/ColumnsCommon.h>
namespace DB
{
namespace ErrorCodes
{
extern const int PARAMETER_OUT_OF_BOUND;
@ -106,7 +110,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]));
@ -179,7 +182,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
@ -246,13 +249,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)
@ -266,9 +269,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()
@ -284,7 +287,7 @@ void ColumnAggregateFunction::insert(const Field & x)
Arena & arena = createOrGetArena();
getData().push_back(arena.alloc(function->sizeOfData()));
getData().push_back(arena.alignedAlloc(function->sizeOfData(), function->alignOfData()));
function->create(getData().back());
ReadBufferFromString read_buffer(x.get<const String &>());
function->deserialize(getData().back(), read_buffer, &arena);
@ -296,7 +299,7 @@ void ColumnAggregateFunction::insertDefault()
Arena & arena = createOrGetArena();
getData().push_back(arena.alloc(function->sizeOfData()));
getData().push_back(arena.alignedAlloc(function->sizeOfData(), function->alignOfData()));
function->create(getData().back());
}
@ -317,7 +320,7 @@ const char * ColumnAggregateFunction::deserializeAndInsertFromArena(const char *
*/
Arena & dst_arena = createOrGetArena();
getData().push_back(dst_arena.alloc(function->sizeOfData()));
getData().push_back(dst_arena.alignedAlloc(function->sizeOfData(), function->alignOfData()));
function->create(getData().back());
/** We will read from src_arena.
@ -411,7 +414,7 @@ void ColumnAggregateFunction::getExtremes(Field & min, Field & max) const
{
/// Place serialized default values into min/max.
PODArrayWithStackMemory<char, 16> place_buffer(func->sizeOfData());
AlignedBuffer place_buffer(func->sizeOfData(), func->alignOfData());
AggregateDataPtr place = place_buffer.data();
String serialized;

View File

@ -1,7 +1,5 @@
#pragma once
#include <Common/Arena.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/IColumn.h>
@ -16,6 +14,10 @@
namespace DB
{
class Arena;
using ArenaPtr = std::shared_ptr<Arena>;
using Arenas = std::vector<ArenaPtr>;
/** Column of states of aggregate functions.
* Presented as an array of pointers to the states of aggregate functions (data).
@ -123,14 +125,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);
}
@ -313,7 +313,8 @@ bool ColumnArray::hasEqualOffsets(const ColumnArray & other) const
const Offsets & offsets1 = getOffsets();
const Offsets & offsets2 = other.getOffsets();
return offsets1.size() == offsets2.size() && 0 == memcmp(&offsets1[0], &offsets2[0], sizeof(offsets1[0]) * offsets1.size());
return offsets1.size() == offsets2.size()
&& (offsets1.size() == 0 || 0 == memcmp(offsets1.data(), offsets2.data(), sizeof(offsets1[0]) * offsets1.size()));
}
@ -662,7 +663,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
{
@ -693,6 +694,9 @@ void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_h
ColumnPtr ColumnArray::replicate(const Offsets & replicate_offsets) const
{
if (replicate_offsets.empty())
return cloneEmpty();
if (typeid_cast<const ColumnUInt8 *>(data.get())) return replicateNumber<UInt8>(replicate_offsets);
if (typeid_cast<const ColumnUInt16 *>(data.get())) return replicateNumber<UInt16>(replicate_offsets);
if (typeid_cast<const ColumnUInt32 *>(data.get())) return replicateNumber<UInt32>(replicate_offsets);
@ -748,8 +752,11 @@ ColumnPtr ColumnArray::replicateNumber(const Offsets & replicate_offsets) const
current_new_offset += value_size;
res_offsets.push_back(current_new_offset);
res_data.resize(res_data.size() + value_size);
memcpy(&res_data[res_data.size() - value_size], &src_data[prev_data_offset], value_size * sizeof(T));
if (value_size)
{
res_data.resize(res_data.size() + value_size);
memcpy(&res_data[res_data.size() - value_size], &src_data[prev_data_offset], value_size * sizeof(T));
}
}
prev_replicate_offset = replicate_offsets[i];
@ -820,10 +827,13 @@ ColumnPtr ColumnArray::replicateString(const Offsets & replicate_offsets) const
prev_src_string_offset_local += chars_size;
}
/// Copies the characters of the array of rows.
res_chars.resize(res_chars.size() + sum_chars_size);
memcpySmallAllowReadWriteOverflow15(
&res_chars[res_chars.size() - sum_chars_size], &src_chars[prev_src_string_offset], sum_chars_size);
if (sum_chars_size)
{
/// Copies the characters of the array of rows.
res_chars.resize(res_chars.size() + sum_chars_size);
memcpySmallAllowReadWriteOverflow15(
&res_chars[res_chars.size() - sum_chars_size], &src_chars[prev_src_string_offset], sum_chars_size);
}
}
prev_replicate_offset = replicate_offsets[i];

View File

@ -91,7 +91,7 @@ MutableColumnPtr ColumnDecimal<T>::cloneResized(size_t size) const
new_col.data.resize(size);
size_t count = std::min(this->size(), size);
memcpy(&new_col.data[0], &data[0], count * sizeof(data[0]));
memcpy(new_col.data.data(), data.data(), count * sizeof(data[0]));
if (size > count)
memset(static_cast<void *>(&new_col.data[count]), static_cast<int>(value_type()), (size - count) * sizeof(value_type));
@ -130,9 +130,9 @@ ColumnPtr ColumnDecimal<T>::filter(const IColumn::Filter & filt, ssize_t result_
if (result_size_hint)
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
const UInt8 * filt_pos = &filt[0];
const UInt8 * filt_pos = filt.data();
const UInt8 * filt_end = filt_pos + size;
const T * data_pos = &data[0];
const T * data_pos = data.data();
while (filt_pos < filt_end)
{

View File

@ -36,7 +36,7 @@ MutableColumnPtr ColumnFixedString::cloneResized(size_t size) const
new_col.chars.resize(size * n);
size_t count = std::min(this->size(), size);
memcpy(&(new_col.chars[0]), &chars[0], count * n * sizeof(chars[0]));
memcpy(new_col.chars.data(), chars.data(), count * n * sizeof(chars[0]));
if (size > count)
memset(&(new_col.chars[count * n]), '\0', (size - count) * n);
@ -165,9 +165,9 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result
if (result_size_hint)
res->chars.reserve(result_size_hint > 0 ? result_size_hint * n : chars.size());
const UInt8 * filt_pos = &filt[0];
const UInt8 * filt_pos = filt.data();
const UInt8 * filt_end = filt_pos + col_size;
const UInt8 * data_pos = &chars[0];
const UInt8 * data_pos = chars.data();
#if __SSE2__
/** A slightly more optimized version.

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

@ -1,5 +1,5 @@
#include <Core/Defines.h>
#include <Common/Arena.h>
#include <Columns/Collator.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsCommon.h>
@ -159,6 +159,36 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const
}
StringRef ColumnString::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
{
size_t string_size = sizeAt(n);
size_t offset = offsetAt(n);
StringRef res;
res.size = sizeof(string_size) + string_size;
char * pos = arena.allocContinue(res.size, begin);
memcpy(pos, &string_size, sizeof(string_size));
memcpy(pos + sizeof(string_size), &chars[offset], string_size);
res.data = pos;
return res;
}
const char * ColumnString::deserializeAndInsertFromArena(const char * pos)
{
const size_t string_size = *reinterpret_cast<const size_t *>(pos);
pos += sizeof(string_size);
const size_t old_size = chars.size();
const size_t new_size = old_size + string_size;
chars.resize(new_size);
memcpy(&chars[old_size], pos, string_size);
offsets.push_back(new_size);
return pos + string_size;
}
ColumnPtr ColumnString::index(const IColumn & indexes, size_t limit) const
{
return selectIndexImpl(*this, indexes, limit);

View File

@ -4,7 +4,6 @@
#include <Columns/IColumn.h>
#include <Common/PODArray.h>
#include <Common/Arena.h>
#include <Common/SipHash.h>
#include <Common/memcpySmall.h>
@ -176,34 +175,9 @@ public:
offsets.resize_assume_reserved(offsets.size() - n);
}
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
{
size_t string_size = sizeAt(n);
size_t offset = offsetAt(n);
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
StringRef res;
res.size = sizeof(string_size) + string_size;
char * pos = arena.allocContinue(res.size, begin);
memcpy(pos, &string_size, sizeof(string_size));
memcpy(pos + sizeof(string_size), &chars[offset], string_size);
res.data = pos;
return res;
}
const char * deserializeAndInsertFromArena(const char * pos) override
{
const size_t string_size = *reinterpret_cast<const size_t *>(pos);
pos += sizeof(string_size);
const size_t old_size = chars.size();
const size_t new_size = old_size + string_size;
chars.resize(new_size);
memcpy(&chars[old_size], pos, string_size);
offsets.push_back(new_size);
return pos + string_size;
}
const char * deserializeAndInsertFromArena(const char * pos) override;
void updateHashWithValue(size_t n, SipHash & hash) const override
{

View File

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

View File

@ -115,7 +115,7 @@ MutableColumnPtr ColumnVector<T>::cloneResized(size_t size) const
new_col.data.resize(size);
size_t count = std::min(this->size(), size);
memcpy(&new_col.data[0], &data[0], count * sizeof(data[0]));
memcpy(new_col.data.data(), data.data(), count * sizeof(data[0]));
if (size > count)
memset(static_cast<void *>(&new_col.data[count]), static_cast<int>(value_type()), (size - count) * sizeof(value_type));
@ -160,9 +160,9 @@ ColumnPtr ColumnVector<T>::filter(const IColumn::Filter & filt, ssize_t result_s
if (result_size_hint)
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
const UInt8 * filt_pos = &filt[0];
const UInt8 * filt_pos = filt.data();
const UInt8 * filt_end = filt_pos + size;
const T * data_pos = &data[0];
const T * data_pos = data.data();
#if __SSE2__
/** A slightly more optimized version.

View File

@ -21,7 +21,7 @@ size_t countBytesInFilter(const IColumn::Filter & filt)
* It would be better to use != 0, then this does not allow SSE2.
*/
const Int8 * pos = reinterpret_cast<const Int8 *>(&filt[0]);
const Int8 * pos = reinterpret_cast<const Int8 *>(filt.data());
const Int8 * end = pos + filt.size();
#if __SSE2__ && __POPCNT__
@ -196,10 +196,10 @@ namespace
res_elems.reserve((result_size_hint * src_elems.size() + size - 1) / size);
}
const UInt8 * filt_pos = &filt[0];
const UInt8 * filt_pos = filt.data();
const auto filt_end = filt_pos + size;
auto offsets_pos = &src_offsets[0];
auto offsets_pos = src_offsets.data();
const auto offsets_begin = offsets_pos;
/// copy array ending at *end_offset_ptr

View File

@ -0,0 +1,49 @@
#include <Common/AlignedBuffer.h>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_ALLOCATE_MEMORY;
}
void AlignedBuffer::alloc(size_t size, size_t alignment)
{
void * new_buf;
int res = ::posix_memalign(&new_buf, std::max(alignment, sizeof(void*)), size);
if (0 != res)
throwFromErrno("Cannot allocate memory (posix_memalign), size: "
+ formatReadableSizeWithBinarySuffix(size) + ", alignment: " + formatReadableSizeWithBinarySuffix(alignment) + ".",
ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
buf = new_buf;
}
void AlignedBuffer::dealloc()
{
if (buf)
::free(buf);
}
void AlignedBuffer::reset(size_t size, size_t alignment)
{
dealloc();
alloc(size, alignment);
}
AlignedBuffer::AlignedBuffer(size_t size, size_t alignment)
{
alloc(size, alignment);
}
AlignedBuffer::~AlignedBuffer()
{
dealloc();
}
}

View File

@ -0,0 +1,36 @@
#pragma once
#include <cstdlib>
#include <utility>
#include <boost/noncopyable.hpp>
namespace DB
{
/** Aligned piece of memory.
* It can only be allocated and destroyed.
* MemoryTracker is not used. It is intended for small pieces of memory.
*/
class AlignedBuffer : private boost::noncopyable
{
private:
void * buf = nullptr;
void alloc(size_t size, size_t alignment);
void dealloc();
public:
AlignedBuffer() {}
AlignedBuffer(size_t size, size_t alignment);
AlignedBuffer(AlignedBuffer && old) { std::swap(buf, old.buf); }
~AlignedBuffer();
void reset(size_t size, size_t alignment);
char * data() { return static_cast<char *>(buf); }
const char * data() const { return static_cast<const char *>(buf); }
};
}

View File

@ -100,6 +100,7 @@ private:
}
friend class ArenaAllocator;
template <size_t> friend class AlignedArenaAllocator;
public:
Arena(size_t initial_size_ = 4096, size_t growth_factor_ = 2, size_t linear_growth_threshold_ = 128 * 1024 * 1024)
@ -124,6 +125,26 @@ public:
return res;
}
/// Get peice of memory with alignment
char * alignedAlloc(size_t size, size_t alignment)
{
do
{
void * head_pos = head->pos;
size_t space = head->end - head->pos;
auto res = static_cast<char *>(std::align(alignment, size, head_pos, space));
if (res)
{
head->pos = static_cast<char *>(head_pos);
head->pos += size;
return res;
}
addChunk(size + alignment);
} while (true);
}
/** Rollback just performed allocation.
* Must pass size not more that was just allocated.
*/
@ -132,7 +153,7 @@ public:
head->pos -= size;
}
/** Begin or expand allocation of contiguous piece of memory.
/** Begin or expand allocation of contiguous piece of memory without alignment.
* 'begin' - current begin of piece of memory, if it need to be expanded, or nullptr, if it need to be started.
* If there is no space in chunk to expand current piece of memory - then copy all piece to new chunk and change value of 'begin'.
* NOTE This method is usable only for latest allocation. For earlier allocations, see 'realloc' method.
@ -159,6 +180,37 @@ public:
return res;
}
char * alignedAllocContinue(size_t size, char const *& begin, size_t alignment)
{
char * res;
do
{
void * head_pos = head->pos;
size_t space = head->end - head->pos;
res = static_cast<char *>(std::align(alignment, size, head_pos, space));
if (res)
{
head->pos = static_cast<char *>(head_pos);
head->pos += size;
break;
}
char * prev_end = head->pos;
addChunk(size + alignment);
if (begin)
begin = alignedInsert(begin, prev_end - begin, alignment);
else
break;
} while (true);
if (!begin)
begin = res;
return res;
}
/// NOTE Old memory region is wasted.
char * realloc(const char * old_data, size_t old_size, size_t new_size)
{
@ -168,6 +220,14 @@ public:
return res;
}
char * alignedRealloc(const char * old_data, size_t old_size, size_t new_size, size_t alignment)
{
char * res = alignedAlloc(new_size, alignment);
if (old_data)
memcpy(res, old_data, old_size);
return res;
}
/// Insert string without alignment.
const char * insert(const char * data, size_t size)
{
@ -176,6 +236,13 @@ public:
return res;
}
const char * alignedInsert(const char * data, size_t size, size_t alignment)
{
char * res = alignedAlloc(size, alignment);
memcpy(res, data, size);
return res;
}
/// Size of chunks in bytes.
size_t size() const
{

View File

@ -38,6 +38,36 @@ public:
};
template <size_t alignment>
class AlignedArenaAllocator
{
public:
static void * alloc(size_t size, Arena * arena)
{
return arena->alignedAlloc(size, alignment);
}
static void * realloc(void * buf, size_t old_size, size_t new_size, Arena * arena)
{
char const * data = reinterpret_cast<char *>(buf);
if (data + old_size == arena->head->pos)
{
arena->alignedAllocContinue(new_size - old_size, data, alignment);
return reinterpret_cast<void *>(const_cast<char *>(data));
}
else
{
return arena->alignedRealloc(data, old_size, new_size, alignment);
}
}
static void free(void * /*buf*/, size_t /*size*/)
{
}
};
/// Switches to ordinary Allocator after REAL_ALLOCATION_TRESHOLD bytes to avoid fragmentation and trash in Arena.
template <size_t REAL_ALLOCATION_TRESHOLD = 4096, typename TRealAllocator = Allocator<false>, typename TArenaAllocator = ArenaAllocator>
class MixedArenaAllocator : private TRealAllocator
@ -72,6 +102,10 @@ public:
};
template <size_t alignment, size_t REAL_ALLOCATION_TRESHOLD = 4096>
using MixedAlignedArenaAllocator = MixedArenaAllocator<REAL_ALLOCATION_TRESHOLD, Allocator<false>, AlignedArenaAllocator<alignment>>;
template <size_t N = 64, typename Base = ArenaAllocator>
class ArenaAllocatorWithStackMemoty : public Base
{

View File

@ -1,13 +1,20 @@
#pragma once
#include <cstddef>
#include <cstdlib>
#include <common/likely.h>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_ALLOCATE_MEMORY;
}
/** An array of (almost) unchangable size:
* the size is specified in the constructor;
* `resize` method removes old data, and necessary only for
@ -20,6 +27,7 @@ namespace DB
* `sizeof` is equal to the size of one pointer.
*
* Not exception-safe.
*
* Copying is supported via assign() method. Moving empties the original object.
* That is, it is inconvenient to use this array in many cases.
*
@ -78,7 +86,7 @@ public:
if (this == &src)
return;
setEmpty();
data = src.data;
data_ptr = src.data_ptr;
src.setEmpty();
}
@ -87,7 +95,7 @@ public:
if (this == &src)
return *this;
uninit();
data = src.data;
data_ptr = src.data_ptr;
src.setEmpty();
return *this;
@ -147,6 +155,16 @@ public:
return elem(i);
}
T * data()
{
return elemPtr(0);
}
const T * data() const
{
return elemPtr(0);
}
/** Get the piece of memory in which the element should be located.
* The function is intended to initialize an element,
* which has not yet been initialized
@ -154,17 +172,17 @@ public:
*/
char * place(size_t i)
{
return data + sizeof(T) * i;
return data_ptr + sizeof(T) * i;
}
using iterator = T *;
using const_iterator = const T *;
iterator begin() { return &elem(0); }
iterator end() { return &elem(size()); }
iterator begin() { return elemPtr(0); }
iterator end() { return elemPtr(size()); }
const_iterator begin() const { return &elem(0); }
const_iterator end() const { return &elem(size()); }
const_iterator begin() const { return elemPtr(0); }
const_iterator end() const { return elemPtr(size()); }
bool operator== (const AutoArray<T> & rhs) const
{
@ -207,47 +225,69 @@ public:
}
private:
char * data;
static constexpr size_t alignment = alignof(T);
/// Bytes allocated to store size of array before data. It is padded to have minimum size as alignment.
/// Padding is at left and the size is stored at right (just before the first data element).
static constexpr size_t prefix_size = std::max(sizeof(size_t), alignment);
char * data_ptr;
size_t & m_size()
{
return reinterpret_cast<size_t *>(data)[-1];
return reinterpret_cast<size_t *>(data_ptr)[-1];
}
size_t m_size() const
{
return reinterpret_cast<const size_t *>(data)[-1];
return reinterpret_cast<const size_t *>(data_ptr)[-1];
}
T * elemPtr(size_t i)
{
return reinterpret_cast<T *>(data_ptr) + i;
}
const T * elemPtr(size_t i) const
{
return reinterpret_cast<const T *>(data_ptr) + i;
}
T & elem(size_t i)
{
return reinterpret_cast<T *>(data)[i];
return *elemPtr(i);
}
const T & elem(size_t i) const
{
return reinterpret_cast<const T *>(data)[i];
return *elemPtr(i);
}
void setEmpty()
{
data = const_cast<char *>(reinterpret_cast<const char *>(&empty_auto_array_helper)) + sizeof(size_t);
data_ptr = const_cast<char *>(reinterpret_cast<const char *>(&empty_auto_array_helper)) + sizeof(size_t);
}
void init(size_t size_, bool dont_init_elems)
void init(size_t new_size, bool dont_init_elems)
{
if (!size_)
if (!new_size)
{
setEmpty();
return;
}
data = new char[size_ * sizeof(T) + sizeof(size_t)];
data += sizeof(size_t);
m_size() = size_;
void * new_data = nullptr;
int res = posix_memalign(&new_data, alignment, prefix_size + new_size * sizeof(T));
if (0 != res)
throwFromErrno("Cannot allocate memory (posix_memalign) " + formatReadableSizeWithBinarySuffix(new_size) + ".",
ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
data_ptr = static_cast<char *>(new_data);
data_ptr += prefix_size;
m_size() = new_size;
if (!dont_init_elems)
for (size_t i = 0; i < size_; ++i)
for (size_t i = 0; i < new_size; ++i)
new (place(i)) T();
}
@ -255,13 +295,13 @@ private:
{
size_t s = size();
if (likely(s))
if (s)
{
for (size_t i = 0; i < s; ++i)
elem(i).~T();
data -= sizeof(size_t);
delete[] data;
data_ptr -= prefix_size;
free(data_ptr);
}
}
};

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

@ -3,68 +3,80 @@
/// Available metrics. Add something here as you wish.
#define APPLY_FOR_METRICS(M) \
M(Query) \
M(Merge) \
M(PartMutation) \
M(ReplicatedFetch) \
M(ReplicatedSend) \
M(ReplicatedChecks) \
M(BackgroundPoolTask) \
M(BackgroundSchedulePoolTask) \
M(DiskSpaceReservedForMerge) \
M(DistributedSend) \
M(QueryPreempted) \
M(TCPConnection) \
M(HTTPConnection) \
M(InterserverConnection) \
M(OpenFileForRead) \
M(OpenFileForWrite) \
M(Read) \
M(Write) \
M(SendExternalTables) \
M(QueryThread) \
M(ReadonlyReplica) \
M(LeaderReplica) \
M(MemoryTracking) \
M(MemoryTrackingInBackgroundProcessingPool) \
M(MemoryTrackingInBackgroundSchedulePool) \
M(MemoryTrackingForMerges) \
M(LeaderElection) \
M(EphemeralNode) \
M(ZooKeeperSession) \
M(ZooKeeperWatch) \
M(ZooKeeperRequest) \
M(DelayedInserts) \
M(ContextLockWait) \
M(StorageBufferRows) \
M(StorageBufferBytes) \
M(DictCacheRequests) \
M(Revision) \
M(RWLockWaitingReaders) \
M(RWLockWaitingWriters) \
M(RWLockActiveReaders) \
M(RWLockActiveWriters)
M(Query, "Number of executing queries") \
M(Merge, "Number of executing background merges") \
M(PartMutation, "Number of mutations (ALTER DELETE/UPDATE)") \
M(ReplicatedFetch, "Number of data parts fetching from replica") \
M(ReplicatedSend, "Number of data parts sending to replicas") \
M(ReplicatedChecks, "Number of data parts checking for consistency") \
M(BackgroundPoolTask, "Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches or replication queue bookkeeping)") \
M(BackgroundSchedulePoolTask, "Number of active tasks in BackgroundSchedulePool. This pool is used for periodic tasks of ReplicatedMergeTree like cleaning old data parts, altering data parts, replica re-initialization, etc.") \
M(DiskSpaceReservedForMerge, "Disk space reserved for currently running background merges. It is slightly more than total size of currently merging parts.") \
M(DistributedSend, "Number of connections sending data, that was INSERTed to Distributed tables, to remote servers. Both synchronous and asynchronous mode.") \
M(QueryPreempted, "Number of queries that are stopped and waiting due to 'priority' setting.") \
M(TCPConnection, "Number of connections to TCP server (clients with native interface)") \
M(HTTPConnection, "Number of connections to HTTP server") \
M(InterserverConnection, "Number of connections from other replicas to fetch parts") \
M(OpenFileForRead, "Number of files open for reading") \
M(OpenFileForWrite, "Number of files open for writing") \
M(Read, "Number of read (read, pread, io_getevents, etc.) syscalls in fly") \
M(Write, "Number of write (write, pwrite, io_getevents, etc.) syscalls in fly") \
M(SendExternalTables, "Number of connections that are sending data for external tables to remote servers. External tables are used to implement GLOBAL IN and GLOBAL JOIN operators with distributed subqueries.") \
M(QueryThread, "Number of query processing threads") \
M(ReadonlyReplica, "Number of Replicated tables that are currently in readonly state due to re-initialization after ZooKeeper session loss or due to startup without ZooKeeper configured.") \
M(LeaderReplica, "Number of Replicated tables that are leaders. Leader replica is responsible for assigning merges, cleaning old blocks for deduplications and a few more bookkeeping tasks. There may be no more than one leader across all replicas at one moment of time. If there is no leader it will be elected soon or it indicate an issue.") \
M(MemoryTracking, "Total amount of memory (bytes) allocated in currently executing queries. Note that some memory allocations may not be accounted.") \
M(MemoryTrackingInBackgroundProcessingPool, "Total amount of memory (bytes) allocated in background processing pool (that is dedicated for backround merges, mutations and fetches). Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \
M(MemoryTrackingInBackgroundSchedulePool, "Total amount of memory (bytes) allocated in background schedule pool (that is dedicated for bookkeeping tasks of Replicated tables).") \
M(MemoryTrackingForMerges, "Total amount of memory (bytes) allocated for background merges. Included in MemoryTrackingInBackgroundProcessingPool. Note that this value may include a drift when the memory was allocated in a context of background processing pool and freed in other context or vice-versa. This happens naturally due to caches for tables indexes and doesn't indicate memory leaks.") \
M(LeaderElection, "Number of Replicas participating in leader election. Equals to total number of replicas in usual cases.") \
M(EphemeralNode, "Number of ephemeral nodes hold in ZooKeeper.") \
M(ZooKeeperSession, "Number of sessions (connections) to ZooKeeper. Should be no more than one, because using more than one connection to ZooKeeper may lead to bugs due to lack of linearizability (stale reads) that ZooKeeper consistency model allows.") \
M(ZooKeeperWatch, "Number of watches (event subscriptions) in ZooKeeper.") \
M(ZooKeeperRequest, "Number of requests to ZooKeeper in fly.") \
M(DelayedInserts, "Number of INSERT queries that are throttled due to high number of active data parts for partition in a MergeTree table.") \
M(ContextLockWait, "Number of threads waiting for lock in Context. This is global lock.") \
M(StorageBufferRows, "Number of rows in buffers of Buffer tables") \
M(StorageBufferBytes, "Number of bytes in buffers of Buffer tables") \
M(DictCacheRequests, "Number of requests in fly to data sources of dictionaries of cache type.") \
M(Revision, "Revision of the server. It is a number incremented for every release or release candidate.") \
M(RWLockWaitingReaders, "Number of threads waiting for read on a table RWLock.") \
M(RWLockWaitingWriters, "Number of threads waiting for write on a table RWLock.") \
M(RWLockActiveReaders, "Number of threads holding read lock in a table RWLock.") \
M(RWLockActiveWriters, "Number of threads holding write lock in a table RWLock.") \
namespace CurrentMetrics
{
#define M(NAME) extern const Metric NAME = __COUNTER__;
#define M(NAME, DOCUMENTATION) extern const Metric NAME = __COUNTER__;
APPLY_FOR_METRICS(M)
#undef M
constexpr Metric END = __COUNTER__;
std::atomic<Value> values[END] {}; /// Global variable, initialized by zeros.
const char * getDescription(Metric event)
const char * getName(Metric event)
{
static const char * descriptions[] =
static const char * strings[] =
{
#define M(NAME) #NAME,
#define M(NAME, DOCUMENTATION) #NAME,
APPLY_FOR_METRICS(M)
#undef M
};
return descriptions[event];
return strings[event];
}
const char * getDocumentation(Metric event)
{
static const char * strings[] =
{
#define M(NAME, DOCUMENTATION) DOCUMENTATION,
APPLY_FOR_METRICS(M)
#undef M
};
return strings[event];
}
Metric end() { return END; }

View File

@ -24,8 +24,10 @@ namespace CurrentMetrics
using Metric = size_t;
using Value = DB::Int64;
/// Get name of metric by identifier. Returns statically allocated string.
const char * getName(Metric event);
/// Get text description of metric by identifier. Returns statically allocated string.
const char * getDescription(Metric event);
const char * getDocumentation(Metric event);
/// Metric identifier -> current value of metric.
extern std::atomic<Value> values[];

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

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

@ -40,7 +40,7 @@ private:
int thread_number = 0;
std::time_t enqueue_time = 0;
std::time_t start_time = 0;
Type type;
Type type = Read;
};
public:

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

@ -57,7 +57,7 @@ namespace UnicodeBar
inline std::string render(double width)
{
std::string res(getWidthInBytes(width), '\0');
render(width, &res[0]);
render(width, res.data());
return res;
}
}

View File

@ -11,7 +11,6 @@
namespace ProfileEvents
{
extern const Event ObsoleteEphemeralNode;
extern const Event LeaderElectionAcquiredLeadership;
}

View File

@ -340,7 +340,7 @@ void read(String & s, ReadBuffer & in)
throw Exception("Too large string size while reading from ZooKeeper", ZMARSHALLINGERROR);
s.resize(size);
in.read(&s[0], size);
in.read(s.data(), size);
}
template <size_t N> void read(std::array<char, N> & s, ReadBuffer & in)
@ -349,7 +349,7 @@ template <size_t N> void read(std::array<char, N> & s, ReadBuffer & in)
read(size, in);
if (size != N)
throw Exception("Unexpected array size while reading from ZooKeeper", ZMARSHALLINGERROR);
in.read(&s[0], N);
in.read(s.data(), N);
}
void read(Stat & stat, ReadBuffer & in)
@ -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

@ -85,7 +85,7 @@ void calcBias(pfHash hash, std::vector<int> & counts, int reps, Rand & r)
hash(&K, keybytes, 0, &A);
int * cursor = &counts[0];
int * cursor = counts.data();
for (int iBit = 0; iBit < keybits; iBit++)
{
@ -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>;
@ -185,7 +186,7 @@ int main(int argc, char ** argv)
std::cerr
<< "arr1.size(): " << arr1.size() << ", arr2.size(): " << arr2.size() << std::endl
<< "&arr1[0]: " << &arr1[0] << ", &arr2[0]: " << &arr2[0] << std::endl
<< "arr1.data(): " << arr1.data() << ", arr2.data(): " << arr2.data() << std::endl
<< "arr1[0]: " << arr1[0] << ", arr2[0]: " << arr2[0] << std::endl;
}

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

@ -53,7 +53,7 @@ int main(int, char **)
watch.stop();
UInt64 check = CityHash_v1_0_2::CityHash64(&hashes[0], hashes.size());
UInt64 check = CityHash_v1_0_2::CityHash64(hashes.data(), hashes.size());
std::cerr << std::fixed << std::setprecision(2)
<< "CityHash64 (check = " << check << ")"
@ -73,12 +73,12 @@ int main(int, char **)
reinterpret_cast<unsigned char *>(&hashes[i * 16]),
reinterpret_cast<const unsigned char *>(strings[i].data()),
strings[i].size(),
reinterpret_cast<const unsigned char *>(&seed[0]));
reinterpret_cast<const unsigned char *>(seed.data()));
}
watch.stop();
UInt64 check = CityHash_v1_0_2::CityHash64(&hashes[0], hashes.size());
UInt64 check = CityHash_v1_0_2::CityHash64(hashes.data(), hashes.size());
std::cerr << std::fixed << std::setprecision(2)
<< "SipHash (check = " << check << ")"
@ -99,7 +99,7 @@ int main(int, char **)
watch.stop();
UInt64 check = CityHash_v1_0_2::CityHash64(&hashes[0], hashes.size());
UInt64 check = CityHash_v1_0_2::CityHash64(hashes.data(), hashes.size());
std::cerr << std::fixed << std::setprecision(2)
<< "SipHash, stream (check = " << check << ")"
@ -121,7 +121,7 @@ int main(int, char **)
watch.stop();
UInt64 check = CityHash_v1_0_2::CityHash64(&hashes[0], hashes.size());
UInt64 check = CityHash_v1_0_2::CityHash64(hashes.data(), hashes.size());
std::cerr << std::fixed << std::setprecision(2)
<< "MD5 (check = " << check << ")"

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);
{
@ -302,18 +299,18 @@ int main(int argc, char ** argv)
setAffinity();
if (!method || method == 1) test<identity> (n, &data[0], "0: identity");
if (!method || method == 2) test<intHash32> (n, &data[0], "1: intHash32");
if (!method || method == 3) test<_intHash64>(n, &data[0], "2: intHash64");
if (!method || method == 4) test<hash3> (n, &data[0], "3: two rounds");
if (!method || method == 5) test<hash4> (n, &data[0], "4: two rounds and two variables");
if (!method || method == 6) test<hash5> (n, &data[0], "5: two rounds with less ops");
if (!method || method == 7) test<murmurMix> (n, &data[0], "6: murmur64 mixer");
if (!method || method == 8) test<mulShift> (n, &data[0], "7: mulShift");
if (!method || method == 9) test<tabulation>(n, &data[0], "8: tabulation");
if (!method || method == 1) test<identity> (n, data.data(), "0: identity");
if (!method || method == 2) test<intHash32> (n, data.data(), "1: intHash32");
if (!method || method == 3) test<_intHash64>(n, data.data(), "2: intHash64");
if (!method || method == 4) test<hash3> (n, data.data(), "3: two rounds");
if (!method || method == 5) test<hash4> (n, data.data(), "4: two rounds and two variables");
if (!method || method == 6) test<hash5> (n, data.data(), "5: two rounds with less ops");
if (!method || method == 7) test<murmurMix> (n, data.data(), "6: murmur64 mixer");
if (!method || method == 8) test<mulShift> (n, data.data(), "7: mulShift");
if (!method || method == 9) test<tabulation>(n, data.data(), "8: tabulation");
#if __x86_64__
if (!method || method == 10) test<crc32Hash> (n, &data[0], "9: crc32");
if (!method || method == 10) test<crc32Hash> (n, data.data(), "9: crc32");
#endif
return 0;

View File

@ -359,7 +359,7 @@ int main(int argc, char ** argv)
DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
DB::CompressedReadBuffer in2(in1);
in2.readStrict(reinterpret_cast<char*>(&data[0]), sizeof(data[0]) * n);
in2.readStrict(reinterpret_cast<char*>(data.data()), sizeof(data[0]) * n);
watch.stop();
std::cerr

View File

@ -262,7 +262,7 @@ int main(int argc, char ** argv)
DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
DB::CompressedReadBuffer in2(in1);
in2.readStrict(reinterpret_cast<char*>(&data[0]), sizeof(data[0]) * n);
in2.readStrict(reinterpret_cast<char*>(data.data()), sizeof(data[0]) * n);
watch.stop();
std::cerr << std::fixed << std::setprecision(2)
@ -500,7 +500,7 @@ int main(int argc, char ** argv)
for (size_t i = 0; i < MapTwoLevel::NUM_BUCKETS; ++i)
pool.schedule(std::bind(merge2,
&maps[0], num_threads, i));
maps.data(), num_threads, i));
pool.wait();
@ -553,8 +553,7 @@ int main(int argc, char ** argv)
watch.restart();
for (size_t i = 0; i < MapTwoLevel::NUM_BUCKETS; ++i)
pool.schedule(std::bind(merge2,
&maps[0], num_threads, i));
pool.schedule(std::bind(merge2, maps.data(), num_threads, i));
pool.wait();
@ -731,7 +730,7 @@ int main(int argc, char ** argv)
pool.schedule(std::bind(aggregate4,
std::ref(local_maps[i]),
std::ref(global_map),
&mutexes[0],
mutexes.data(),
data.begin() + (data.size() * i) / num_threads,
data.begin() + (data.size() * (i + 1)) / num_threads));

View File

@ -301,7 +301,7 @@ int main(int argc, char ** argv)
DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
DB::CompressedReadBuffer in2(in1);
in2.readStrict(reinterpret_cast<char*>(&data[0]), sizeof(data[0]) * n);
in2.readStrict(reinterpret_cast<char*>(data.data()), sizeof(data[0]) * n);
watch.stop();
std::cerr << std::fixed << std::setprecision(2)

View File

@ -71,9 +71,9 @@ int main(int argc, char ** argv)
{
Stopwatch watch;
if (method == 1) sort1(&data[0], n);
if (method == 2) sort2(&data[0], n);
if (method == 3) sort3(&data[0], n);
if (method == 1) sort1(data.data(), n);
if (method == 2) sort2(data.data(), n);
if (method == 3) sort3(data.data(), n);
watch.stop();
double elapsed = watch.elapsedSeconds();

View File

@ -415,5 +415,52 @@ inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b)
return a >= b;
}
}
namespace DB
{
template <typename A, typename B> struct EqualsOp
{
/// An operation that gives the same result, if arguments are passed in reverse order.
using SymmetricOp = EqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::equalsOp(a, b); }
};
template <typename A, typename B> struct NotEqualsOp
{
using SymmetricOp = NotEqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::notEqualsOp(a, b); }
};
template <typename A, typename B> struct GreaterOp;
template <typename A, typename B> struct LessOp
{
using SymmetricOp = GreaterOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::lessOp(a, b); }
};
template <typename A, typename B> struct GreaterOp
{
using SymmetricOp = LessOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::greaterOp(a, b); }
};
template <typename A, typename B> struct GreaterOrEqualsOp;
template <typename A, typename B> struct LessOrEqualsOp
{
using SymmetricOp = GreaterOrEqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::lessOrEqualsOp(a, b); }
};
template <typename A, typename B> struct GreaterOrEqualsOp
{
using SymmetricOp = LessOrEqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::greaterOrEqualsOp(a, b); }
};
}

View File

@ -0,0 +1,311 @@
#pragma once
#include <common/arithmeticOverflow.h>
#include <Core/Block.h>
#include <Core/AccurateComparison.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
#include <Functions/FunctionHelpers.h> /// TODO Core should not depend on Functions
namespace DB
{
namespace ErrorCodes
{
extern const int DECIMAL_OVERFLOW;
}
///
inline bool allowDecimalComparison(const IDataType & left_type, const IDataType & right_type)
{
if (isDecimal(left_type))
{
if (isDecimal(right_type) || notDecimalButComparableToDecimal(right_type))
return true;
}
else if (notDecimalButComparableToDecimal(left_type) && isDecimal(right_type))
return true;
return false;
}
template <size_t > struct ConstructDecInt { using Type = Int32; };
template <> struct ConstructDecInt<8> { using Type = Int64; };
template <> struct ConstructDecInt<16> { using Type = Int128; };
template <typename T, typename U>
struct DecCompareInt
{
using Type = typename ConstructDecInt<(!IsDecimalNumber<U> || sizeof(T) > sizeof(U)) ? sizeof(T) : sizeof(U)>::Type;
using TypeA = Type;
using TypeB = Type;
};
///
template <typename A, typename B, template <typename, typename> typename Operation, bool _check_overflow = true,
bool _actual = IsDecimalNumber<A> || IsDecimalNumber<B>>
class DecimalComparison
{
public:
using CompareInt = typename DecCompareInt<A, B>::Type;
using Op = Operation<CompareInt, CompareInt>;
using ColVecA = std::conditional_t<IsDecimalNumber<A>, ColumnDecimal<A>, ColumnVector<A>>;
using ColVecB = std::conditional_t<IsDecimalNumber<B>, ColumnDecimal<B>, ColumnVector<B>>;
using ArrayA = typename ColVecA::Container;
using ArrayB = typename ColVecB::Container;
DecimalComparison(Block & block, size_t result, const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right)
{
if (!apply(block, result, col_left, col_right))
throw Exception("Wrong decimal comparison with " + col_left.type->getName() + " and " + col_right.type->getName(),
ErrorCodes::LOGICAL_ERROR);
}
static bool apply(Block & block, size_t result [[maybe_unused]],
const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right)
{
if constexpr (_actual)
{
ColumnPtr c_res;
Shift shift = getScales<A, B>(col_left.type, col_right.type);
c_res = applyWithScale(col_left.column, col_right.column, shift);
if (c_res)
block.getByPosition(result).column = std::move(c_res);
return true;
}
return false;
}
static bool compare(A a, B b, UInt32 scale_a, UInt32 scale_b)
{
static const UInt32 max_scale = maxDecimalPrecision<Decimal128>();
if (scale_a > max_scale || scale_b > max_scale)
throw Exception("Bad scale of decimal field", ErrorCodes::DECIMAL_OVERFLOW);
Shift shift;
if (scale_a < scale_b)
shift.a = DataTypeDecimal<B>(maxDecimalPrecision<B>(), scale_b).getScaleMultiplier(scale_b - scale_a);
if (scale_a > scale_b)
shift.b = DataTypeDecimal<A>(maxDecimalPrecision<A>(), scale_a).getScaleMultiplier(scale_a - scale_b);
return applyWithScale(a, b, shift);
}
private:
struct Shift
{
CompareInt a = 1;
CompareInt b = 1;
bool none() const { return a == 1 && b == 1; }
bool left() const { return a != 1; }
bool right() const { return b != 1; }
};
template <typename T, typename U>
static auto applyWithScale(T a, U b, const Shift & shift)
{
if (shift.left())
return apply<true, false>(a, b, shift.a);
else if (shift.right())
return apply<false, true>(a, b, shift.b);
return apply<false, false>(a, b, 1);
}
template <typename T, typename U>
static std::enable_if_t<IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
getScales(const DataTypePtr & left_type, const DataTypePtr & right_type)
{
const DataTypeDecimal<T> * decimal0 = checkDecimal<T>(*left_type);
const DataTypeDecimal<U> * decimal1 = checkDecimal<U>(*right_type);
Shift shift;
if (decimal0 && decimal1)
{
auto result_type = decimalResultType(*decimal0, *decimal1, false, false);
shift.a = result_type.scaleFactorFor(*decimal0, false);
shift.b = result_type.scaleFactorFor(*decimal1, false);
}
else if (decimal0)
shift.b = decimal0->getScaleMultiplier();
else if (decimal1)
shift.a = decimal1->getScaleMultiplier();
return shift;
}
template <typename T, typename U>
static std::enable_if_t<IsDecimalNumber<T> && !IsDecimalNumber<U>, Shift>
getScales(const DataTypePtr & left_type, const DataTypePtr &)
{
Shift shift;
const DataTypeDecimal<T> * decimal0 = checkDecimal<T>(*left_type);
if (decimal0)
shift.b = decimal0->getScaleMultiplier();
return shift;
}
template <typename T, typename U>
static std::enable_if_t<!IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
getScales(const DataTypePtr &, const DataTypePtr & right_type)
{
Shift shift;
const DataTypeDecimal<U> * decimal1 = checkDecimal<U>(*right_type);
if (decimal1)
shift.a = decimal1->getScaleMultiplier();
return shift;
}
template <bool scale_left, bool scale_right>
static ColumnPtr apply(const ColumnPtr & c0, const ColumnPtr & c1, CompareInt scale)
{
auto c_res = ColumnUInt8::create();
if constexpr (_actual)
{
bool c0_is_const = c0->isColumnConst();
bool c1_is_const = c1->isColumnConst();
if (c0_is_const && c1_is_const)
{
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
A a = c0_const->template getValue<A>();
B b = c1_const->template getValue<B>();
UInt8 res = apply<scale_left, scale_right>(a, b, scale);
return DataTypeUInt8().createColumnConst(c0->size(), toField(res));
}
ColumnUInt8::Container & vec_res = c_res->getData();
vec_res.resize(c0->size());
if (c0_is_const)
{
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
A a = c0_const->template getValue<A>();
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
constant_vector<scale_left, scale_right>(a, c1_vec->getData(), vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
else if (c1_is_const)
{
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
B b = c1_const->template getValue<B>();
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
vector_constant<scale_left, scale_right>(c0_vec->getData(), b, vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
else
{
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
{
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
vector_vector<scale_left, scale_right>(c0_vec->getData(), c1_vec->getData(), vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
}
return c_res;
}
template <bool scale_left, bool scale_right>
static NO_INLINE UInt8 apply(A a, B b, CompareInt scale [[maybe_unused]])
{
CompareInt x = a;
CompareInt y = b;
if constexpr (_check_overflow)
{
bool overflow = false;
if constexpr (sizeof(A) > sizeof(CompareInt))
overflow |= (A(x) != a);
if constexpr (sizeof(B) > sizeof(CompareInt))
overflow |= (B(y) != b);
if constexpr (std::is_unsigned_v<A>)
overflow |= (x < 0);
if constexpr (std::is_unsigned_v<B>)
overflow |= (y < 0);
if constexpr (scale_left)
overflow |= common::mulOverflow(x, scale, x);
if constexpr (scale_right)
overflow |= common::mulOverflow(y, scale, y);
if (overflow)
throw Exception("Can't compare", ErrorCodes::DECIMAL_OVERFLOW);
}
else
{
if constexpr (scale_left)
x *= scale;
if constexpr (scale_right)
y *= scale;
}
return Op::apply(x, y);
}
template <bool scale_left, bool scale_right>
static void NO_INLINE vector_vector(const ArrayA & a, const ArrayB & b, PaddedPODArray<UInt8> & c,
CompareInt scale)
{
size_t size = a.size();
const A * a_pos = a.data();
const B * b_pos = b.data();
UInt8 * c_pos = c.data();
const A * a_end = a_pos + size;
while (a_pos < a_end)
{
*c_pos = apply<scale_left, scale_right>(*a_pos, *b_pos, scale);
++a_pos;
++b_pos;
++c_pos;
}
}
template <bool scale_left, bool scale_right>
static void NO_INLINE vector_constant(const ArrayA & a, B b, PaddedPODArray<UInt8> & c, CompareInt scale)
{
size_t size = a.size();
const A * a_pos = a.data();
UInt8 * c_pos = c.data();
const A * a_end = a_pos + size;
while (a_pos < a_end)
{
*c_pos = apply<scale_left, scale_right>(*a_pos, b, scale);
++a_pos;
++c_pos;
}
}
template <bool scale_left, bool scale_right>
static void NO_INLINE constant_vector(A a, const ArrayB & b, PaddedPODArray<UInt8> & c, CompareInt scale)
{
size_t size = b.size();
const B * b_pos = b.data();
UInt8 * c_pos = c.data();
const B * b_end = b_pos + size;
while (b_pos < b_end)
{
*c_pos = apply<scale_left, scale_right>(a, *b_pos, scale);
++b_pos;
++c_pos;
}
}
};
}

View File

@ -4,8 +4,8 @@
#include <IO/WriteHelpers.h>
#include <Core/Field.h>
#include <Core/DecimalComparison.h>
#include <Common/FieldVisitors.h>
#include <Functions/FunctionsComparison.h>
namespace DB

View File

@ -19,11 +19,15 @@ GraphiteRollupSortedBlockInputStream::GraphiteRollupSortedBlockInputStream(
params(params), time_of_merge(time_of_merge)
{
size_t max_size_of_aggregate_state = 0;
for (const auto & pattern : params.patterns)
if (pattern.function->sizeOfData() > max_size_of_aggregate_state)
max_size_of_aggregate_state = pattern.function->sizeOfData();
size_t max_alignment_of_aggregate_state = 1;
place_for_aggregate_state.resize(max_size_of_aggregate_state);
for (const auto & pattern : params.patterns)
{
max_size_of_aggregate_state = std::max(max_size_of_aggregate_state, pattern.function->sizeOfData());
max_alignment_of_aggregate_state = std::max(max_alignment_of_aggregate_state, pattern.function->alignOfData());
}
place_for_aggregate_state.reset(max_size_of_aggregate_state, max_alignment_of_aggregate_state);
/// Memoize column numbers in block.
path_column_num = header.getPositionByName(params.path_column_name);

View File

@ -8,6 +8,7 @@
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnAggregateFunction.h>
#include <Common/OptimizedRegularExpression.h>
#include <Common/AlignedBuffer.h>
namespace DB
@ -186,7 +187,7 @@ private:
time_t current_time_rounded = 0;
const Graphite::Pattern * current_pattern = nullptr;
std::vector<char> place_for_aggregate_state;
AlignedBuffer place_for_aggregate_state;
bool aggregate_state_created = false; /// Invariant: if true then current_pattern is not NULL.
const Graphite::Pattern * selectPatternForPath(StringRef path) const;

View File

@ -2,6 +2,7 @@
#include <Core/Row.h>
#include <Core/ColumnNumbers.h>
#include <Common/AlignedBuffer.h>
#include <DataStreams/MergingSortedBlockInputStream.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
@ -74,7 +75,7 @@ private:
IAggregateFunction::AddFunc add_function = nullptr;
std::vector<size_t> column_numbers;
MutableColumnPtr merged_column;
std::vector<char> state;
AlignedBuffer state;
bool created = false;
/// In case when column has type AggregateFunction: use the aggregate function from itself instead of 'function' above.
@ -84,7 +85,7 @@ private:
{
function = AggregateFunctionFactory::instance().get(function_name, argument_types);
add_function = function->getAddressOfAddFunction();
state.resize(function->sizeOfData());
state.reset(function->sizeOfData(), function->alignOfData());
}
void createState()

View File

@ -4,6 +4,7 @@
#include <Columns/ColumnAggregateFunction.h>
#include <Columns/FilterDescription.h>
#include <Common/typeid_cast.h>
#include <Common/Arena.h>
namespace DB
@ -34,7 +35,7 @@ TotalsHavingBlockInputStream::TotalsHavingBlockInputStream(
IAggregateFunction * function = column->getAggregateFunction().get();
auto target = ColumnAggregateFunction::create(column->getAggregateFunction(), Arenas(1, arena));
AggregateDataPtr data = arena->alloc(function->sizeOfData());
AggregateDataPtr data = arena->alignedAlloc(function->sizeOfData(), function->alignOfData());
function->create(data);
target->getData().push_back(data);
current_totals.emplace_back(std::move(target));

View File

@ -1,12 +1,14 @@
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Common/Arena.h>
namespace DB
{
class Arena;
using ArenaPtr = std::shared_ptr<Arena>;
class ExpressionActions;

View File

@ -6,6 +6,7 @@
#include <Columns/ColumnAggregateFunction.h>
#include <Common/typeid_cast.h>
#include <Common/AlignedBuffer.h>
#include <Formats/FormatSettings.h>
#include <DataTypes/DataTypeAggregateFunction.h>
@ -68,7 +69,7 @@ void DataTypeAggregateFunction::deserializeBinary(Field & field, ReadBuffer & is
field = String();
String & s = get<String &>(field);
s.resize(size);
istr.readStrict(&s[0], size);
istr.readStrict(s.data(), size);
}
void DataTypeAggregateFunction::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
@ -82,7 +83,7 @@ void DataTypeAggregateFunction::deserializeBinary(IColumn & column, ReadBuffer &
Arena & arena = column_concrete.createOrGetArena();
size_t size_of_state = function->sizeOfData();
AggregateDataPtr place = arena.alloc(size_of_state);
AggregateDataPtr place = arena.alignedAlloc(size_of_state, function->alignOfData());
function->create(place);
try
@ -123,13 +124,14 @@ void DataTypeAggregateFunction::deserializeBinaryBulk(IColumn & column, ReadBuff
vec.reserve(vec.size() + limit);
size_t size_of_state = function->sizeOfData();
size_t align_of_state = function->alignOfData();
for (size_t i = 0; i < limit; ++i)
{
if (istr.eof())
break;
AggregateDataPtr place = arena.alloc(size_of_state);
AggregateDataPtr place = arena.alignedAlloc(size_of_state, align_of_state);
function->create(place);
@ -160,7 +162,7 @@ static void deserializeFromString(const AggregateFunctionPtr & function, IColumn
Arena & arena = column_concrete.createOrGetArena();
size_t size_of_state = function->sizeOfData();
AggregateDataPtr place = arena.alloc(size_of_state);
AggregateDataPtr place = arena.alignedAlloc(size_of_state, function->alignOfData());
function->create(place);
@ -257,7 +259,7 @@ Field DataTypeAggregateFunction::getDefault() const
{
Field field = String();
PODArrayWithStackMemory<char, 16> place_buffer(function->sizeOfData());
AlignedBuffer place_buffer(function->sizeOfData(), function->alignOfData());
AggregateDataPtr place = place_buffer.data();
function->create(place);

View File

@ -52,7 +52,7 @@ void DataTypeFixedString::deserializeBinary(Field & field, ReadBuffer & istr) co
field = String();
String & s = get<String &>(field);
s.resize(n);
istr.readStrict(&s[0], n);
istr.readStrict(s.data(), n);
}
@ -88,7 +88,8 @@ void DataTypeFixedString::serializeBinaryBulk(const IColumn & column, WriteBuffe
if (limit == 0 || offset + limit > size)
limit = size - offset;
ostr.write(reinterpret_cast<const char *>(&data[n * offset]), n * limit);
if (limit)
ostr.write(reinterpret_cast<const char *>(&data[n * offset]), n * limit);
}

View File

@ -1,4 +1,6 @@
#include <DataTypes/DataTypeFunction.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
namespace DB
@ -6,22 +8,24 @@ namespace DB
std::string DataTypeFunction::getName() const
{
std::string res = "Function(";
WriteBufferFromOwnString res;
res << "Function(";
if (argument_types.size() > 1)
res += "(";
res << "(";
for (size_t i = 0; i < argument_types.size(); ++i)
{
if (i > 0)
res += ", ";
res << ", ";
const DataTypePtr & type = argument_types[i];
res += type ? type->getName() : "?";
res << (type ? type->getName() : "?");
}
if (argument_types.size() > 1)
res += ")";
res += " -> ";
res += return_type ? return_type->getName() : "?";
res += ")";
return res;
res << ")";
res << " -> ";
res << (return_type ? return_type->getName() : "?");
res << ")";
return res.str();
}
bool DataTypeFunction::equals(const IDataType & rhs) const

View File

@ -220,7 +220,8 @@ void DataTypeNumberBase<T>::serializeBinaryBulk(const IColumn & column, WriteBuf
if (limit == 0 || offset + limit > size)
limit = size - offset;
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(typename ColumnVector<T>::value_type) * limit);
if (limit)
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(typename ColumnVector<T>::value_type) * limit);
}
template <typename T>

View File

@ -38,7 +38,7 @@ void DataTypeString::deserializeBinary(Field & field, ReadBuffer & istr) const
field = String();
String & s = get<String &>(field);
s.resize(size);
istr.readStrict(&s[0], size);
istr.readStrict(s.data(), size);
}
@ -96,7 +96,7 @@ void DataTypeString::serializeBinaryBulk(const IColumn & column, WriteBuffer & o
{
UInt64 str_size = offsets[0] - 1;
writeVarUInt(str_size, ostr);
ostr.write(reinterpret_cast<const char *>(&data[0]), str_size);
ostr.write(reinterpret_cast<const char *>(data.data()), str_size);
++offset;
}

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

@ -7,6 +7,7 @@
#include <Core/Block.h>
#include <Core/ColumnNumbers.h>
namespace DB
{

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;
}
@ -1691,9 +1690,9 @@ struct DivideIntegralByConstantImpl
libdivide::divider<A> divider(b);
size_t size = a.size();
const A * a_pos = &a[0];
const A * a_pos = a.data();
const A * a_end = a_pos + size;
ResultType * c_pos = &c[0];
ResultType * c_pos = c.data();
#if __SSE2__
static constexpr size_t values_per_sse_register = 16 / sizeof(A);
@ -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

@ -9,6 +9,7 @@
#include <Functions/GatherUtils/GatherUtils.h>
#include <Common/HashTable/HashMap.h>
#include <Common/HashTable/ClearableHashMap.h>
#include <Common/AlignedBuffer.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ASTExpressionList.h>
@ -2779,8 +2780,8 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName
void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
IAggregateFunction & agg_func = *aggregate_function.get();
std::unique_ptr<char[]> place_holder { new char[agg_func.sizeOfData()] };
AggregateDataPtr place = place_holder.get();
AlignedBuffer place_holder(agg_func.sizeOfData(), agg_func.alignOfData());
AggregateDataPtr place = place_holder.data();
std::unique_ptr<Arena> arena = agg_func.allocatesMemoryInArena() ? std::make_unique<Arena>() : nullptr;
@ -3574,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

@ -110,7 +110,7 @@ public:
vec_res.resize(size * (IPV6_MAX_TEXT_LENGTH + 1));
offsets_res.resize(size);
auto begin = reinterpret_cast<char *>(&vec_res[0]);
auto begin = reinterpret_cast<char *>(vec_res.data());
auto pos = begin;
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i)
@ -219,7 +219,7 @@ public:
vec_res.resize(size * (IPV6_MAX_TEXT_LENGTH + 1));
offsets_res.resize(size);
auto begin = reinterpret_cast<char *>(&vec_res[0]);
auto begin = reinterpret_cast<char *>(vec_res.data());
auto pos = begin;
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i)
@ -519,7 +519,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!checkDataType<DataTypeUInt32>(&*arguments[0]))
if (!checkDataType<DataTypeUInt32>(arguments[0].get()))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt32",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -543,7 +543,7 @@ public:
vec_res.resize(vec_in.size() * (IPV4_MAX_TEXT_LENGTH + 1)); /// the longest value is: 255.255.255.255\0
offsets_res.resize(vec_in.size());
char * begin = reinterpret_cast<char *>(&vec_res[0]);
char * begin = reinterpret_cast<char *>(vec_res.data());
char * pos = begin;
for (size_t i = 0; i < vec_in.size(); ++i)
@ -714,7 +714,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!checkDataType<DataTypeUInt64>(&*arguments[0]))
if (!checkDataType<DataTypeUInt64>(arguments[0].get()))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -1120,7 +1120,7 @@ public:
size_t size = input_rows_count;
vec_to.resize(size);
Rand64Impl::execute(reinterpret_cast<UInt64 *>(&vec_to[0]), vec_to.size() * 2);
Rand64Impl::execute(reinterpret_cast<UInt64 *>(vec_to.data()), vec_to.size() * 2);
for (UInt128 & uuid: vec_to)
{
@ -1154,10 +1154,10 @@ public:
if (!arguments[0]->isString()
&& !arguments[0]->isFixedString()
&& !arguments[0]->isDateOrDateTime()
&& !checkDataType<DataTypeUInt8>(&*arguments[0])
&& !checkDataType<DataTypeUInt16>(&*arguments[0])
&& !checkDataType<DataTypeUInt32>(&*arguments[0])
&& !checkDataType<DataTypeUInt64>(&*arguments[0]))
&& !checkDataType<DataTypeUInt8>(arguments[0].get())
&& !checkDataType<DataTypeUInt16>(arguments[0].get())
&& !checkDataType<DataTypeUInt32>(arguments[0].get())
&& !checkDataType<DataTypeUInt64>(arguments[0].get()))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -1259,7 +1259,7 @@ public:
out_offsets.resize(size);
out_vec.resize(in_vec.size() * 2 - size);
char * begin = reinterpret_cast<char *>(&out_vec[0]);
char * begin = reinterpret_cast<char *>(out_vec.data());
char * pos = begin;
size_t prev_offset = 0;
@ -1303,7 +1303,7 @@ public:
out_offsets.resize(size);
out_vec.resize(in_vec.size() * 2 + size);
char * begin = reinterpret_cast<char *>(&out_vec[0]);
char * begin = reinterpret_cast<char *>(out_vec.data());
char * pos = begin;
size_t n = col_fstr_in->getN();
@ -1415,7 +1415,7 @@ public:
out_offsets.resize(size);
out_vec.resize(in_vec.size() / 2 + size);
char * begin = reinterpret_cast<char *>(&out_vec[0]);
char * begin = reinterpret_cast<char *>(out_vec.data());
char * pos = begin;
size_t prev_offset = 0;
@ -1569,7 +1569,7 @@ public:
out_offsets.resize(size);
out_vec.resize(in_vec.size());
char * begin = reinterpret_cast<char *>(&out_vec[0]);
char * begin = reinterpret_cast<char *>(out_vec.data());
char * pos = begin;
ColumnString::Offset current_in_offset = 0;
@ -1616,9 +1616,9 @@ public:
out_offsets.resize(size);
out_vec.resize(in_vec.size() + size);
char * begin = reinterpret_cast<char *>(&out_vec[0]);
char * begin = reinterpret_cast<char *>(out_vec.data());
char * pos = begin;
const char * pos_in = reinterpret_cast<const char *>(&in_vec[0]);
const char * pos_in = reinterpret_cast<const char *>(in_vec.data());
size_t n = col_fstr_in->getN();

View File

@ -1,76 +1,23 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsComparison.h>
#include <Common/typeid_cast.h>
namespace DB
{
void registerFunctionEquals(FunctionFactory & factory);
void registerFunctionNotEquals(FunctionFactory & factory);
void registerFunctionLess(FunctionFactory & factory);
void registerFunctionGreater(FunctionFactory & factory);
void registerFunctionLessOrEquals(FunctionFactory & factory);
void registerFunctionGreaterOrEquals(FunctionFactory & factory);
void registerFunctionsComparison(FunctionFactory & factory)
{
factory.registerFunction<FunctionEquals>();
factory.registerFunction<FunctionNotEquals>();
factory.registerFunction<FunctionLess>();
factory.registerFunction<FunctionGreater>();
factory.registerFunction<FunctionLessOrEquals>();
factory.registerFunction<FunctionGreaterOrEquals>();
}
template <>
void FunctionComparison<EqualsOp, NameEquals>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleEqualityImpl<FunctionComparison<EqualsOp, NameEquals>, FunctionAnd>(block, result, x, y,
tuple_size, input_rows_count);
}
template <>
void FunctionComparison<NotEqualsOp, NameNotEquals>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleEqualityImpl<FunctionComparison<NotEqualsOp, NameNotEquals>, FunctionOr>(block, result, x, y,
tuple_size, input_rows_count);
}
template <>
void FunctionComparison<LessOp, NameLess>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleLessGreaterImpl<
FunctionComparison<LessOp, NameLess>,
FunctionComparison<LessOp, NameLess>>(block, result, x, y, tuple_size, input_rows_count);
}
template <>
void FunctionComparison<GreaterOp, NameGreater>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleLessGreaterImpl<
FunctionComparison<GreaterOp, NameGreater>,
FunctionComparison<GreaterOp, NameGreater>>(block, result, x, y, tuple_size, input_rows_count);
}
template <>
void FunctionComparison<LessOrEqualsOp, NameLessOrEquals>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleLessGreaterImpl<
FunctionComparison<LessOp, NameLess>,
FunctionComparison<LessOrEqualsOp, NameLessOrEquals>>(block, result, x, y, tuple_size, input_rows_count);
}
template <>
void FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleLessGreaterImpl<
FunctionComparison<GreaterOp, NameGreater>,
FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>>(block, result, x, y, tuple_size, input_rows_count);
registerFunctionEquals(factory);
registerFunctionNotEquals(factory);
registerFunctionLess(factory);
registerFunctionGreater(factory);
registerFunctionLessOrEquals(factory);
registerFunctionGreaterOrEquals(factory);
}
}

View File

@ -9,7 +9,6 @@
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeString.h>
@ -27,6 +26,8 @@
#include <Functions/FunctionHelpers.h>
#include <Core/AccurateComparison.h>
#include <Core/DecimalComparison.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/ReadHelpers.h>
@ -37,11 +38,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int DECIMAL_OVERFLOW;
}
/** Comparison functions: ==, !=, <, >, <=, >=.
* The comparison functions always return 0 or 1 (UInt8).
@ -55,94 +51,8 @@ namespace ErrorCodes
* - tuples (lexicographic comparison).
*
* Exception: You can compare the date and datetime with a constant string. Example: EventDate = '2015-01-01'.
*
* TODO Arrays.
*/
template <typename A, typename B> struct EqualsOp
{
/// An operation that gives the same result, if arguments are passed in reverse order.
using SymmetricOp = EqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::equalsOp(a, b); }
#if USE_EMBEDDED_COMPILER
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool /*is_signed*/)
{
return x->getType()->isIntegerTy() ? b.CreateICmpEQ(x, y) : b.CreateFCmpOEQ(x, y); /// qNaNs always compare false
}
#endif
};
template <typename A, typename B> struct NotEqualsOp
{
using SymmetricOp = NotEqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::notEqualsOp(a, b); }
#if USE_EMBEDDED_COMPILER
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool /*is_signed*/)
{
return x->getType()->isIntegerTy() ? b.CreateICmpNE(x, y) : b.CreateFCmpONE(x, y);
}
#endif
};
template <typename A, typename B> struct GreaterOp;
template <typename A, typename B> struct LessOp
{
using SymmetricOp = GreaterOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::lessOp(a, b); }
#if USE_EMBEDDED_COMPILER
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
{
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSLT(x, y) : b.CreateICmpULT(x, y)) : b.CreateFCmpOLT(x, y);
}
#endif
};
template <typename A, typename B> struct GreaterOp
{
using SymmetricOp = LessOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::greaterOp(a, b); }
#if USE_EMBEDDED_COMPILER
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
{
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSGT(x, y) : b.CreateICmpUGT(x, y)) : b.CreateFCmpOGT(x, y);
}
#endif
};
template <typename A, typename B> struct GreaterOrEqualsOp;
template <typename A, typename B> struct LessOrEqualsOp
{
using SymmetricOp = GreaterOrEqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::lessOrEqualsOp(a, b); }
#if USE_EMBEDDED_COMPILER
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
{
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSLE(x, y) : b.CreateICmpULE(x, y)) : b.CreateFCmpOLE(x, y);
}
#endif
};
template <typename A, typename B> struct GreaterOrEqualsOp
{
using SymmetricOp = LessOrEqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::greaterOrEqualsOp(a, b); }
#if USE_EMBEDDED_COMPILER
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
{
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSGE(x, y) : b.CreateICmpUGE(x, y)) : b.CreateFCmpOGE(x, y);
}
#endif
};
template <typename A, typename B, typename Op>
struct NumComparisonImpl
@ -156,9 +66,9 @@ struct NumComparisonImpl
*/
size_t size = a.size();
const A * a_pos = &a[0];
const B * b_pos = &b[0];
UInt8 * c_pos = &c[0];
const A * a_pos = a.data();
const B * b_pos = b.data();
UInt8 * c_pos = c.data();
const A * a_end = a_pos + size;
while (a_pos < a_end)
@ -173,8 +83,8 @@ struct NumComparisonImpl
static void NO_INLINE vector_constant(const PaddedPODArray<A> & a, B b, PaddedPODArray<UInt8> & c)
{
size_t size = a.size();
const A * a_pos = &a[0];
UInt8 * c_pos = &c[0];
const A * a_pos = a.data();
UInt8 * c_pos = c.data();
const A * a_end = a_pos + size;
while (a_pos < a_end)
@ -196,296 +106,6 @@ struct NumComparisonImpl
}
};
///
inline bool allowDecimalComparison(const IDataType & left_type, const IDataType & right_type)
{
if (isDecimal(left_type))
{
if (isDecimal(right_type) || notDecimalButComparableToDecimal(right_type))
return true;
}
else if (notDecimalButComparableToDecimal(left_type) && isDecimal(right_type))
return true;
return false;
}
template <size_t > struct ConstructDecInt { using Type = Int32; };
template <> struct ConstructDecInt<8> { using Type = Int64; };
template <> struct ConstructDecInt<16> { using Type = Int128; };
template <typename T, typename U>
struct DecCompareInt
{
using Type = typename ConstructDecInt<(!IsDecimalNumber<U> || sizeof(T) > sizeof(U)) ? sizeof(T) : sizeof(U)>::Type;
using TypeA = Type;
using TypeB = Type;
};
///
template <typename A, typename B, template <typename, typename> typename Operation, bool _check_overflow = true,
bool _actual = IsDecimalNumber<A> || IsDecimalNumber<B>>
class DecimalComparison
{
public:
using CompareInt = typename DecCompareInt<A, B>::Type;
using Op = Operation<CompareInt, CompareInt>;
using ColVecA = std::conditional_t<IsDecimalNumber<A>, ColumnDecimal<A>, ColumnVector<A>>;
using ColVecB = std::conditional_t<IsDecimalNumber<B>, ColumnDecimal<B>, ColumnVector<B>>;
using ArrayA = typename ColVecA::Container;
using ArrayB = typename ColVecB::Container;
DecimalComparison(Block & block, size_t result, const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right)
{
if (!apply(block, result, col_left, col_right))
throw Exception("Wrong decimal comparison with " + col_left.type->getName() + " and " + col_right.type->getName(),
ErrorCodes::LOGICAL_ERROR);
}
static bool apply(Block & block, size_t result [[maybe_unused]],
const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right)
{
if constexpr (_actual)
{
ColumnPtr c_res;
Shift shift = getScales<A, B>(col_left.type, col_right.type);
c_res = applyWithScale(col_left.column, col_right.column, shift);
if (c_res)
block.getByPosition(result).column = std::move(c_res);
return true;
}
return false;
}
static bool compare(A a, B b, UInt32 scale_a, UInt32 scale_b)
{
static const UInt32 max_scale = maxDecimalPrecision<Decimal128>();
if (scale_a > max_scale || scale_b > max_scale)
throw Exception("Bad scale of decimal field", ErrorCodes::DECIMAL_OVERFLOW);
Shift shift;
if (scale_a < scale_b)
shift.a = DataTypeDecimal<B>(maxDecimalPrecision<B>(), scale_b).getScaleMultiplier(scale_b - scale_a);
if (scale_a > scale_b)
shift.b = DataTypeDecimal<A>(maxDecimalPrecision<A>(), scale_a).getScaleMultiplier(scale_a - scale_b);
return applyWithScale(a, b, shift);
}
private:
struct Shift
{
CompareInt a = 1;
CompareInt b = 1;
bool none() const { return a == 1 && b == 1; }
bool left() const { return a != 1; }
bool right() const { return b != 1; }
};
template <typename T, typename U>
static auto applyWithScale(T a, U b, const Shift & shift)
{
if (shift.left())
return apply<true, false>(a, b, shift.a);
else if (shift.right())
return apply<false, true>(a, b, shift.b);
return apply<false, false>(a, b, 1);
}
template <typename T, typename U>
static std::enable_if_t<IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
getScales(const DataTypePtr & left_type, const DataTypePtr & right_type)
{
const DataTypeDecimal<T> * decimal0 = checkDecimal<T>(*left_type);
const DataTypeDecimal<U> * decimal1 = checkDecimal<U>(*right_type);
Shift shift;
if (decimal0 && decimal1)
{
auto result_type = decimalResultType(*decimal0, *decimal1, false, false);
shift.a = result_type.scaleFactorFor(*decimal0, false);
shift.b = result_type.scaleFactorFor(*decimal1, false);
}
else if (decimal0)
shift.b = decimal0->getScaleMultiplier();
else if (decimal1)
shift.a = decimal1->getScaleMultiplier();
return shift;
}
template <typename T, typename U>
static std::enable_if_t<IsDecimalNumber<T> && !IsDecimalNumber<U>, Shift>
getScales(const DataTypePtr & left_type, const DataTypePtr &)
{
Shift shift;
const DataTypeDecimal<T> * decimal0 = checkDecimal<T>(*left_type);
if (decimal0)
shift.b = decimal0->getScaleMultiplier();
return shift;
}
template <typename T, typename U>
static std::enable_if_t<!IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
getScales(const DataTypePtr &, const DataTypePtr & right_type)
{
Shift shift;
const DataTypeDecimal<U> * decimal1 = checkDecimal<U>(*right_type);
if (decimal1)
shift.a = decimal1->getScaleMultiplier();
return shift;
}
template <bool scale_left, bool scale_right>
static ColumnPtr apply(const ColumnPtr & c0, const ColumnPtr & c1, CompareInt scale)
{
auto c_res = ColumnUInt8::create();
if constexpr (_actual)
{
bool c0_const = c0->isColumnConst();
bool c1_const = c1->isColumnConst();
if (c0_const && c1_const)
{
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
A a = c0_const->template getValue<A>();
B b = c1_const->template getValue<B>();
UInt8 res = apply<scale_left, scale_right>(a, b, scale);
return DataTypeUInt8().createColumnConst(c0->size(), toField(res));
}
ColumnUInt8::Container & vec_res = c_res->getData();
vec_res.resize(c0->size());
if (c0_const)
{
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
A a = c0_const->template getValue<A>();
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
constant_vector<scale_left, scale_right>(a, c1_vec->getData(), vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
else if (c1_const)
{
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
B b = c1_const->template getValue<B>();
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
vector_constant<scale_left, scale_right>(c0_vec->getData(), b, vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
else
{
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
{
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
vector_vector<scale_left, scale_right>(c0_vec->getData(), c1_vec->getData(), vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
}
return c_res;
}
template <bool scale_left, bool scale_right>
static NO_INLINE UInt8 apply(A a, B b, CompareInt scale [[maybe_unused]])
{
CompareInt x = a;
CompareInt y = b;
if constexpr (_check_overflow)
{
bool overflow = false;
if constexpr (sizeof(A) > sizeof(CompareInt))
overflow |= (A(x) != a);
if constexpr (sizeof(B) > sizeof(CompareInt))
overflow |= (B(y) != b);
if constexpr (std::is_unsigned_v<A>)
overflow |= (x < 0);
if constexpr (std::is_unsigned_v<B>)
overflow |= (y < 0);
if constexpr (scale_left)
overflow |= common::mulOverflow(x, scale, x);
if constexpr (scale_right)
overflow |= common::mulOverflow(y, scale, y);
if (overflow)
throw Exception("Can't compare", ErrorCodes::DECIMAL_OVERFLOW);
}
else
{
if constexpr (scale_left)
x *= scale;
if constexpr (scale_right)
y *= scale;
}
return Op::apply(x, y);
}
template <bool scale_left, bool scale_right>
static void NO_INLINE vector_vector(const ArrayA & a, const ArrayB & b, PaddedPODArray<UInt8> & c,
CompareInt scale)
{
size_t size = a.size();
const A * a_pos = &a[0];
const B * b_pos = &b[0];
UInt8 * c_pos = &c[0];
const A * a_end = a_pos + size;
while (a_pos < a_end)
{
*c_pos = apply<scale_left, scale_right>(*a_pos, *b_pos, scale);
++a_pos;
++b_pos;
++c_pos;
}
}
template <bool scale_left, bool scale_right>
static void NO_INLINE vector_constant(const ArrayA & a, B b, PaddedPODArray<UInt8> & c, CompareInt scale)
{
size_t size = a.size();
const A * a_pos = &a[0];
UInt8 * c_pos = &c[0];
const A * a_end = a_pos + size;
while (a_pos < a_end)
{
*c_pos = apply<scale_left, scale_right>(*a_pos, b, scale);
++a_pos;
++c_pos;
}
}
template <bool scale_left, bool scale_right>
static void NO_INLINE constant_vector(A a, const ArrayB & b, PaddedPODArray<UInt8> & c, CompareInt scale)
{
size_t size = b.size();
const B * b_pos = &b[0];
UInt8 * c_pos = &c[0];
const B * b_end = b_pos + size;
while (b_pos < b_end)
{
*c_pos = apply<scale_left, scale_right>(a, *b_pos, scale);
++b_pos;
++c_pos;
}
}
};
inline int memcmp16(const void * a, const void * b)
{
@ -531,7 +151,7 @@ struct StringComparisonImpl
{
a_size = a_offsets[0];
b_size = b_offsets[0];
res = memcmp(&a_data[0], &b_data[0], std::min(a_size, b_size));
res = memcmp(a_data.data(), b_data.data(), std::min(a_size, b_size));
}
else
{
@ -554,7 +174,7 @@ struct StringComparisonImpl
{
if (i == 0)
{
int res = memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0] - 1, b_n));
int res = memcmp(a_data.data(), b_data.data(), std::min(a_offsets[0] - 1, b_n));
c[i] = Op::apply(res, 0) || (res == 0 && Op::apply(a_offsets[0], b_n + 1));
}
else
@ -579,7 +199,7 @@ struct StringComparisonImpl
/// Trailing zero byte of the smaller string is included in the comparison.
if (i == 0)
{
int res = memcmp(&a_data[0], b_data, std::min(a_offsets[0], b_size));
int res = memcmp(a_data.data(), b_data, std::min(a_offsets[0], b_size));
c[i] = Op::apply(res, 0) || (res == 0 && Op::apply(a_offsets[0], b_size));
}
else
@ -709,7 +329,7 @@ struct StringEqualsImpl
size_t size = a_offsets.size();
for (size_t i = 0; i < size; ++i)
c[i] = positive == ((i == 0)
? (a_offsets[0] == b_offsets[0] && !memcmp(&a_data[0], &b_data[0], a_offsets[0] - 1))
? (a_offsets[0] == b_offsets[0] && !memcmp(a_data.data(), b_data.data(), a_offsets[0] - 1))
: (a_offsets[i] - a_offsets[i - 1] == b_offsets[i] - b_offsets[i - 1]
&& !memcmp(&a_data[a_offsets[i - 1]], &b_data[b_offsets[i - 1]], a_offsets[i] - a_offsets[i - 1] - 1)));
}
@ -722,7 +342,7 @@ struct StringEqualsImpl
size_t size = a_offsets.size();
for (size_t i = 0; i < size; ++i)
c[i] = positive == ((i == 0)
? (a_offsets[0] == b_n + 1 && !memcmp(&a_data[0], &b_data[0], b_n))
? (a_offsets[0] == b_n + 1 && !memcmp(a_data.data(), b_data.data(), b_n))
: (a_offsets[i] - a_offsets[i - 1] == b_n + 1
&& !memcmp(&a_data[a_offsets[i - 1]], &b_data[b_n * i], b_n)));
}
@ -737,7 +357,7 @@ struct StringEqualsImpl
const UInt8 * b_data = reinterpret_cast<const UInt8 *>(b.data());
for (size_t i = 0; i < size; ++i)
c[i] = positive == ((i == 0)
? (a_offsets[0] == b_n + 1 && !memcmp(&a_data[0], b_data, b_n))
? (a_offsets[0] == b_n + 1 && !memcmp(a_data.data(), b_data, b_n))
: (a_offsets[i] - a_offsets[i - 1] == b_n + 1
&& !memcmp(&a_data[a_offsets[i - 1]], b_data, b_n)));
}
@ -904,6 +524,61 @@ struct GenericComparisonImpl
};
#if USE_EMBEDDED_COMPILER
template <template <typename, typename> typename Op> struct CompileOp;
template <> struct CompileOp<EqualsOp>
{
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool /*is_signed*/)
{
return x->getType()->isIntegerTy() ? b.CreateICmpEQ(x, y) : b.CreateFCmpOEQ(x, y); /// qNaNs always compare false
}
};
template <> struct CompileOp<NotEqualsOp>
{
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool /*is_signed*/)
{
return x->getType()->isIntegerTy() ? b.CreateICmpNE(x, y) : b.CreateFCmpONE(x, y);
}
};
template <> struct CompileOp<LessOp>
{
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
{
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSLT(x, y) : b.CreateICmpULT(x, y)) : b.CreateFCmpOLT(x, y);
}
};
template <> struct CompileOp<GreaterOp>
{
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
{
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSGT(x, y) : b.CreateICmpUGT(x, y)) : b.CreateFCmpOGT(x, y);
}
};
template <> struct CompileOp<LessOrEqualsOp>
{
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
{
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSLE(x, y) : b.CreateICmpULE(x, y)) : b.CreateFCmpOLE(x, y);
}
};
template <> struct CompileOp<GreaterOrEqualsOp>
{
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
{
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSGE(x, y) : b.CreateICmpUGE(x, y)) : b.CreateFCmpOGE(x, y);
}
};
#endif
struct NameEquals { static constexpr auto name = "equals"; };
struct NameNotEquals { static constexpr auto name = "notEquals"; };
struct NameLess { static constexpr auto name = "less"; };
@ -944,13 +619,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 +648,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 +683,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()
@ -1585,18 +1260,10 @@ public:
x = nativeCast(b, types[0], x, common);
y = nativeCast(b, types[1], y, common);
}
auto * result = Op<int, int>::compile(b, x, y, typeIsSigned(*types[0]) || typeIsSigned(*types[1]));
auto * result = CompileOp<Op>::compile(b, x, y, typeIsSigned(*types[0]) || typeIsSigned(*types[1]));
return b.CreateSelect(result, b.getInt8(1), b.getInt8(0));
}
#endif
};
using FunctionEquals = FunctionComparison<EqualsOp, NameEquals>;
using FunctionNotEquals = FunctionComparison<NotEqualsOp, NameNotEquals>;
using FunctionLess = FunctionComparison<LessOp, NameLess>;
using FunctionGreater = FunctionComparison<GreaterOp, NameGreater>;
using FunctionLessOrEquals = FunctionComparison<LessOrEqualsOp, NameLessOrEquals>;
using FunctionGreaterOrEquals = FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>;
}

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

@ -744,7 +744,7 @@ public:
nested_types[i] = array_type->getNestedType();
}
const DataTypeFunction * function_type = checkAndGetDataType<DataTypeFunction>(&*arguments[0]);
const DataTypeFunction * function_type = checkAndGetDataType<DataTypeFunction>(arguments[0].get());
if (!function_type || function_type->getArgumentTypes().size() != nested_types.size())
throw Exception("First argument for this overload of " + getName() + " must be a function with "
+ toString(nested_types.size()) + " arguments. Found "
@ -816,7 +816,7 @@ public:
if (arguments.size() == 1)
{
const auto array_type = checkAndGetDataType<DataTypeArray>(&*arguments[0].type);
const auto array_type = checkAndGetDataType<DataTypeArray>(arguments[0].type.get());
if (!array_type)
throw Exception("The only argument for function " + getName() + " must be array. Found "
@ -849,7 +849,7 @@ public:
throw Exception("Expression for function " + getName() + " must return UInt8, found "
+ return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto first_array_type = checkAndGetDataType<DataTypeArray>(&*arguments[1].type);
const auto first_array_type = checkAndGetDataType<DataTypeArray>(arguments[1].type.get());
return Impl::getReturnType(return_type, first_array_type->getNestedType());
}

View File

@ -16,6 +16,7 @@
#include <Common/UnicodeBar.h>
#include <Common/UTF8Helpers.h>
#include <Common/FieldVisitors.h>
#include <Common/AlignedBuffer.h>
#include <Common/config_version.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeArray.h>
@ -607,11 +608,11 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!checkDataType<DataTypeFloat64>(&*arguments[0]) && !checkDataType<DataTypeFloat32>(&*arguments[0])
&& !checkDataType<DataTypeUInt64>(&*arguments[0])
&& !checkDataType<DataTypeUInt32>(&*arguments[0])
&& !checkDataType<DataTypeUInt16>(&*arguments[0])
&& !checkDataType<DataTypeUInt8>(&*arguments[0]))
if (!checkDataType<DataTypeFloat64>(arguments[0].get()) && !checkDataType<DataTypeFloat32>(arguments[0].get())
&& !checkDataType<DataTypeUInt64>(arguments[0].get())
&& !checkDataType<DataTypeUInt32>(arguments[0].get())
&& !checkDataType<DataTypeUInt16>(arguments[0].get())
&& !checkDataType<DataTypeUInt8>(arguments[0].get()))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -927,7 +928,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const DataTypeArray * arr = checkAndGetDataType<DataTypeArray>(&*arguments[0]);
const DataTypeArray * arr = checkAndGetDataType<DataTypeArray>(arguments[0].get());
if (!arr)
throw Exception("Argument for function " + getName() + " must be Array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -954,7 +955,7 @@ FunctionPtr FunctionReplicate::create(const Context &)
DataTypePtr FunctionReplicate::getReturnTypeImpl(const DataTypes & arguments) const
{
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(&*arguments[1]);
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
if (!array_type)
throw Exception("Second argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -1104,7 +1105,7 @@ private:
{
Float64 width = UnicodeBar::getWidth(src, min, max, max_width);
dst_chars.resize(UnicodeBar::getWidthInBytes(width));
UnicodeBar::render(width, &dst_chars[0]);
UnicodeBar::render(width, dst_chars.data());
}
template <typename T>
@ -1401,7 +1402,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const DataTypeAggregateFunction * type = checkAndGetDataType<DataTypeAggregateFunction>(&*arguments[0]);
const DataTypeAggregateFunction * type = checkAndGetDataType<DataTypeAggregateFunction>(arguments[0].get());
if (!type)
throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -1422,14 +1423,9 @@ public:
AggregateFunctionPtr aggregate_function_ptr = column_with_states->getAggregateFunction();
const IAggregateFunction & agg_func = *aggregate_function_ptr;
auto deleter = [&agg_func](char * ptr)
{
agg_func.destroy(ptr);
free(ptr);
};
std::unique_ptr<char, decltype(deleter)> place{reinterpret_cast<char *>(malloc(agg_func.sizeOfData())), deleter};
agg_func.create(place.get()); /// Not much exception-safe. If an exception is thrown out, destroy will be called in vain.
AlignedBuffer place(agg_func.sizeOfData(), agg_func.alignOfData());
agg_func.create(place.data());
SCOPE_EXIT(agg_func.destroy(place.data()));
std::unique_ptr<Arena> arena = agg_func.allocatesMemoryInArena() ? std::make_unique<Arena>() : nullptr;
@ -1441,8 +1437,8 @@ public:
for (const auto & state_to_add : states)
{
/// Will pass empty arena if agg_func does not allocate memory in arena
agg_func.merge(place.get(), state_to_add, arena.get());
agg_func.insertResultInto(place.get(), result_column);
agg_func.merge(place.data(), state_to_add, arena.get());
agg_func.insertResultInto(place.data(), result_column);
}
block.getByPosition(result).column = std::move(result_column_ptr);
@ -1647,7 +1643,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const DataTypeAggregateFunction * type = checkAndGetDataType<DataTypeAggregateFunction>(&*arguments[0]);
const DataTypeAggregateFunction * type = checkAndGetDataType<DataTypeAggregateFunction>(arguments[0].get());
if (!type)
throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

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

@ -1,6 +1,5 @@
#include <Functions/FunctionsNull.h>
#include <Functions/FunctionsLogical.h>
#include <Functions/FunctionsComparison.h>
#include <Functions/FunctionsConditional.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypesNumber.h>
@ -313,7 +312,11 @@ void FunctionNullIf::executeImpl(Block & block, const ColumnNumbers & arguments,
size_t res_pos = temp_block.columns();
temp_block.insert({nullptr, std::make_shared<DataTypeUInt8>(), ""});
FunctionEquals{context}.execute(temp_block, {arguments[0], arguments[1]}, res_pos, input_rows_count);
{
auto equals_func = FunctionFactory::instance().get("equals", context)->build(
{block.getByPosition(arguments[0]), block.getByPosition(arguments[1])});
equals_func->execute(temp_block, {arguments[0], arguments[1]}, res_pos, input_rows_count);
}
/// Argument corresponding to the NULL value.
size_t null_pos = temp_block.columns();

View File

@ -157,7 +157,7 @@ public:
size_t size = input_rows_count;
vec_to.resize(size);
Impl::execute(&vec_to[0], vec_to.size());
Impl::execute(vec_to.data(), vec_to.size());
block.getByPosition(result).column = std::move(col_to);
}
@ -202,7 +202,7 @@ public:
{
is_initialized = true;
typename ColumnVector<ToType>::Container vec_to(1);
Impl::execute(&vec_to[0], vec_to.size());
Impl::execute(vec_to.data(), vec_to.size());
value = vec_to[0];
}

Some files were not shown because too many files have changed in this diff Show More