Merge remote-tracking branch 'origin/master' into add-ext-dict-cassandra

# Conflicts:
#	dbms/tests/integration/image/Dockerfile
This commit is contained in:
Oleg Favstov 2019-04-11 11:21:12 +02:00
commit 95b812f518
244 changed files with 5756 additions and 1447 deletions

View File

@ -1,3 +1,8 @@
## ClickHouse release 19.4.2.7, 2019-03-30
### Bug Fixes
* Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [#4850](https://github.com/yandex/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai))
## ClickHouse release 19.4.1.3, 2019-03-19
### Bug Fixes

View File

@ -178,7 +178,7 @@ include (cmake/use_libcxx.cmake)
# This is intended for more control of what we are linking.
set (DEFAULT_LIBS "")
if (OS_LINUX AND NOT UNBUNDLED)
if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_LIBCXX))
# Note: this probably has no effect, but I'm not an expert in CMake.
set (CMAKE_C_IMPLICIT_LINK_LIBRARIES "")
set (CMAKE_CXX_IMPLICIT_LINK_LIBRARIES "")

View File

@ -10,7 +10,3 @@ ClickHouse is an open-source column-oriented database management system that all
* [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events.
* [Contacts](https://clickhouse.yandex/#contacts) can help to get your questions answered if there are any.
* You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person.
## Upcoming Events
* [ClickHouse Community Meetup](https://www.eventbrite.com/e/clickhouse-meetup-in-madrid-registration-55376746339) in Madrid on April 2.

View File

@ -284,6 +284,7 @@ endif ()
if (USE_INTERNAL_BROTLI_LIBRARY)
add_subdirectory(brotli-cmake)
target_compile_definitions(brotli PRIVATE BROTLI_BUILD_PORTABLE=1)
endif ()
if (USE_INTERNAL_PROTOBUF_LIBRARY)

View File

@ -20,7 +20,7 @@ set (CONFIG_VERSION ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config_version.h)
set (CONFIG_COMMON ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config.h)
include (cmake/version.cmake)
message (STATUS "Will build ${VERSION_FULL} revision ${VERSION_REVISION}")
message (STATUS "Will build ${VERSION_FULL} revision ${VERSION_REVISION} ${VERSION_OFFICIAL}")
configure_file (src/Common/config.h.in ${CONFIG_COMMON})
configure_file (src/Common/config_version.h.in ${CONFIG_VERSION})

View File

@ -1,11 +1,11 @@
# This strings autochanged from release_lib.sh:
set(VERSION_REVISION 54417)
set(VERSION_REVISION 54418)
set(VERSION_MAJOR 19)
set(VERSION_MINOR 5)
set(VERSION_MINOR 6)
set(VERSION_PATCH 1)
set(VERSION_GITHASH 628ed349c335b79a441a1bd6e4bc791d61dfe62c)
set(VERSION_DESCRIBE v19.5.1.1-testing)
set(VERSION_STRING 19.5.1.1)
set(VERSION_GITHASH 30d3496c36cf3945c9828ac0b7cf7d1774a9f845)
set(VERSION_DESCRIBE v19.6.1.1-testing)
set(VERSION_STRING 19.6.1.1)
# end of autochange
set(VERSION_EXTRA "" CACHE STRING "")
@ -24,3 +24,7 @@ set (VERSION_FULL "${VERSION_NAME} ${VERSION_STRING}")
set (VERSION_SO "${VERSION_STRING}")
math (EXPR VERSION_INTEGER "${VERSION_PATCH} + ${VERSION_MINOR}*1000 + ${VERSION_MAJOR}*1000000")
if(YANDEX_OFFICIAL_BUILD)
set(VERSION_OFFICIAL " (official build)")
endif()

View File

@ -797,14 +797,33 @@ private:
written_progress_chars = 0;
written_first_block = false;
{
/// Temporarily apply query settings to context.
std::optional<Settings> old_settings;
SCOPE_EXIT({ if (old_settings) context.setSettings(*old_settings); });
auto apply_query_settings = [&](const IAST & settings_ast)
{
if (!old_settings)
old_settings.emplace(context.getSettingsRef());
for (const auto & change : settings_ast.as<ASTSetQuery>()->changes)
context.setSetting(change.name, change.value);
};
const auto * insert = parsed_query->as<ASTInsertQuery>();
if (insert && insert->settings_ast)
apply_query_settings(*insert->settings_ast);
/// FIXME: try to prettify this cast using `as<>()`
const auto * with_output = dynamic_cast<const ASTQueryWithOutput *>(parsed_query.get());
if (with_output && with_output->settings_ast)
apply_query_settings(*with_output->settings_ast);
connection->forceConnected();
/// INSERT query for which data transfer is needed (not an INSERT SELECT) is processed separately.
const auto * insert_query = parsed_query->as<ASTInsertQuery>();
if (insert_query && !insert_query->select)
if (insert && !insert->select)
processInsertQuery();
else
processOrdinaryQuery();
}
/// Do not change context (current DB, settings) in case of an exception.
if (!got_exception)
@ -964,8 +983,6 @@ private:
{
if (!insert->format.empty())
current_format = insert->format;
if (insert->settings_ast)
InterpreterSetQuery(insert->settings_ast, context).executeForCurrentContext();
}
BlockInputStreamPtr block_input = context.getInputFormat(
@ -1248,10 +1265,6 @@ private:
const auto & id = query_with_output->format->as<ASTIdentifier &>();
current_format = id.name;
}
if (query_with_output->settings_ast)
{
InterpreterSetQuery(query_with_output->settings_ast, context).executeForCurrentContext();
}
}
if (has_vertical_output_suffix)
@ -1510,7 +1523,7 @@ private:
void showClientVersion()
{
std::cout << DBMS_NAME << " client version " << VERSION_STRING << "." << std::endl;
std::cout << DBMS_NAME << " client version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl;
}
public:

View File

@ -1,6 +1,7 @@
#include <iostream>
#include <optional>
#include <boost/program_options.hpp>
#include <boost/algorithm/string/join.hpp>
#include <Common/Exception.h>
#include <IO/WriteBufferFromFileDescriptor.h>
@ -9,6 +10,8 @@
#include <Compression/CompressedReadBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Compression/CompressionFactory.h>
@ -64,7 +67,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv)
("hc", "use LZ4HC instead of LZ4")
("zstd", "use ZSTD instead of LZ4")
("codec", boost::program_options::value<std::vector<std::string>>()->multitoken(), "use codecs combination instead of LZ4")
("level", boost::program_options::value<std::vector<int>>()->multitoken(), "compression levels for codecs specified via --codec")
("level", boost::program_options::value<int>(), "compression level for codecs spicified via flags")
("none", "use no compression instead of LZ4")
("stat", "print block statistics of compressed data")
;
@ -94,6 +97,9 @@ int mainEntryClickHouseCompressor(int argc, char ** argv)
if ((use_lz4hc || use_zstd || use_none) && !codecs.empty())
throw DB::Exception("Wrong options, codec flags like --zstd and --codec options are mutually exclusive", DB::ErrorCodes::BAD_ARGUMENTS);
if (!codecs.empty() && options.count("level"))
throw DB::Exception("Wrong options, --level is not compatible with --codec list", DB::ErrorCodes::BAD_ARGUMENTS);
std::string method_family = "LZ4";
if (use_lz4hc)
@ -103,28 +109,22 @@ int mainEntryClickHouseCompressor(int argc, char ** argv)
else if (use_none)
method_family = "NONE";
std::vector<int> levels;
std::optional<int> level = std::nullopt;
if (options.count("level"))
levels = options["level"].as<std::vector<int>>();
level = options["level"].as<int>();
DB::CompressionCodecPtr codec;
if (!codecs.empty())
{
if (levels.size() > codecs.size())
throw DB::Exception("Specified more levels than codecs", DB::ErrorCodes::BAD_ARGUMENTS);
DB::ParserCodec codec_parser;
std::vector<DB::CodecNameWithLevel> codec_names;
for (size_t i = 0; i < codecs.size(); ++i)
{
if (i < levels.size())
codec_names.emplace_back(codecs[i], levels[i]);
else
codec_names.emplace_back(codecs[i], std::nullopt);
}
codec = DB::CompressionCodecFactory::instance().get(codec_names);
std::string codecs_line = boost::algorithm::join(codecs, ",");
auto ast = DB::parseQuery(codec_parser, "(" + codecs_line + ")", 0);
codec = DB::CompressionCodecFactory::instance().get(ast, nullptr);
}
else
codec = DB::CompressionCodecFactory::instance().get(method_family, levels.empty() ? std::nullopt : std::optional<int>(levels.back()));
codec = DB::CompressionCodecFactory::instance().get(method_family, level);
DB::ReadBufferFromFileDescriptor rb(STDIN_FILENO);

View File

@ -17,11 +17,11 @@ $ ./clickhouse-compressor --decompress < input_file > output_file
Compress data with ZSTD at level 5:
```
$ ./clickhouse-compressor --codec ZSTD --level 5 < input_file > output_file
$ ./clickhouse-compressor --codec 'ZSTD(5)' < input_file > output_file
```
Compress data with ZSTD level 10, LZ4HC level 7 and LZ4.
Compress data with Delta of four bytes and ZSTD level 10.
```
$ ./clickhouse-compressor --codec ZSTD --level 5 --codec LZ4HC --level 7 --codec LZ4 < input_file > output_file
$ ./clickhouse-compressor --codec 'Delta(4)' --codec 'ZSTD(10)' < input_file > output_file
```

View File

@ -1,7 +1,6 @@
#include "ClusterCopier.h"
#include <chrono>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/Logger.h>
#include <Poco/ConsoleChannel.h>
@ -13,14 +12,11 @@
#include <Poco/FileChannel.h>
#include <Poco/SplitterChannel.h>
#include <Poco/Util/HelpFormatter.h>
#include <boost/algorithm/string.hpp>
#include <pcg_random.hpp>
#include <common/logger_useful.h>
#include <Common/ThreadPool.h>
#include <daemon/OwnPatternFormatter.h>
#include <Common/Exception.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
@ -61,6 +57,7 @@
#include <DataStreams/NullBlockOutputStream.h>
#include <IO/Operators.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadBufferFromFile.h>
#include <Functions/registerFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
@ -500,9 +497,6 @@ static ASTPtr extractPartitionKey(const ASTPtr & storage_ast)
ErrorCodes::BAD_ARGUMENTS);
}
ASTPtr arguments_ast = engine.arguments->clone();
ASTs & arguments = arguments_ast->children;
if (isExtendedDefinitionStorage(storage_ast))
{
if (storage.partition_by)
@ -516,6 +510,12 @@ static ASTPtr extractPartitionKey(const ASTPtr & storage_ast)
bool is_replicated = startsWith(engine.name, "Replicated");
size_t min_args = is_replicated ? 3 : 1;
if (!engine.arguments)
throw Exception("Expected arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS);
ASTPtr arguments_ast = engine.arguments->clone();
ASTs & arguments = arguments_ast->children;
if (arguments.size() < min_args)
throw Exception("Expected at least " + toString(min_args) + " arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS);
@ -894,6 +894,28 @@ public:
}
}
void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force)
{
auto local_task_description_path = task_path + "/description";
String task_config_str;
{
ReadBufferFromFile in(task_file);
readStringUntilEOF(task_config_str, in);
}
if (task_config_str.empty())
return;
auto zookeeper = context.getZooKeeper();
zookeeper->createAncestors(local_task_description_path);
auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent);
if (code && force)
zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent);
LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")");
}
void reloadTaskDescription()
{
auto zookeeper = context.getZooKeeper();
@ -1201,7 +1223,8 @@ protected:
auto new_columns_list = std::make_shared<ASTColumns>();
new_columns_list->set(new_columns_list->columns, new_columns);
new_columns_list->set(new_columns_list->indices, query_ast->as<ASTCreateQuery>()->columns_list->indices->clone());
if (auto indices = query_ast->as<ASTCreateQuery>()->columns_list->indices)
new_columns_list->set(new_columns_list->indices, indices->clone());
new_query.replace(new_query.columns_list, new_columns_list);
@ -2103,6 +2126,10 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options)
options.addOption(Poco::Util::Option("task-path", "", "path to task in ZooKeeper")
.argument("task-path").binding("task-path"));
options.addOption(Poco::Util::Option("task-file", "", "path to task file for uploading in ZooKeeper to task-path")
.argument("task-file").binding("task-file"));
options.addOption(Poco::Util::Option("task-upload-force", "", "Force upload task-file even node already exists")
.argument("task-upload-force").binding("task-upload-force"));
options.addOption(Poco::Util::Option("safe-mode", "", "disables ALTER DROP PARTITION in case of errors")
.binding("safe-mode"));
options.addOption(Poco::Util::Option("copy-fault-probability", "", "the copying fails with specified probability (used to test partition state recovering)")
@ -2153,6 +2180,11 @@ void ClusterCopierApp::mainImpl()
auto copier = std::make_unique<ClusterCopier>(task_path, host_id, default_database, *context);
copier->setSafeMode(is_safe_mode);
copier->setCopyFaultProbability(copy_fault_probability);
auto task_file = config().getString("task-file", "");
if (!task_file.empty())
copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false));
copier->init();
copier->process();
}

View File

@ -369,7 +369,7 @@ void LocalServer::setupUsers()
static void showClientVersion()
{
std::cout << DBMS_NAME << " client version " << VERSION_STRING << "." << '\n';
std::cout << DBMS_NAME << " client version " << VERSION_STRING << VERSION_OFFICIAL << "." << '\n';
}
std::string LocalServer::getHelpHeader() const

View File

@ -296,7 +296,7 @@ void HTTPHandler::processQuery(
/// The client can pass a HTTP header indicating supported compression method (gzip or deflate).
String http_response_compression_methods = request.get("Accept-Encoding", "");
bool client_supports_http_compression = false;
ZlibCompressionMethod http_response_compression_method {};
CompressionMethod http_response_compression_method {};
if (!http_response_compression_methods.empty())
{
@ -305,12 +305,17 @@ void HTTPHandler::processQuery(
if (std::string::npos != http_response_compression_methods.find("gzip"))
{
client_supports_http_compression = true;
http_response_compression_method = ZlibCompressionMethod::Gzip;
http_response_compression_method = CompressionMethod::Gzip;
}
else if (std::string::npos != http_response_compression_methods.find("deflate"))
{
client_supports_http_compression = true;
http_response_compression_method = ZlibCompressionMethod::Zlib;
http_response_compression_method = CompressionMethod::Zlib;
}
else if (http_response_compression_methods == "br")
{
client_supports_http_compression = true;
http_response_compression_method = CompressionMethod::Brotli;
}
}
@ -394,11 +399,11 @@ void HTTPHandler::processQuery(
{
if (http_request_compression_method_str == "gzip")
{
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, ZlibCompressionMethod::Gzip);
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, CompressionMethod::Gzip);
}
else if (http_request_compression_method_str == "deflate")
{
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, ZlibCompressionMethod::Zlib);
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, CompressionMethod::Zlib);
}
#if USE_BROTLI
else if (http_request_compression_method_str == "br")

View File

@ -132,7 +132,7 @@ int Server::run()
}
if (config().hasOption("version"))
{
std::cout << DBMS_NAME << " server version " << VERSION_STRING << "." << std::endl;
std::cout << DBMS_NAME << " server version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl;
return 0;
}
return Application::run();

View File

@ -0,0 +1,85 @@
#include <AggregateFunctions/AggregateFunctionLeastSqr.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionLeastSqr(
const String & name,
const DataTypes & arguments,
const Array & params
)
{
assertNoParameters(name, params);
assertBinary(name, arguments);
const IDataType * x_arg = arguments.front().get();
WhichDataType which_x {
x_arg
};
const IDataType * y_arg = arguments.back().get();
WhichDataType which_y {
y_arg
};
#define FOR_LEASTSQR_TYPES_2(M, T) \
M(T, UInt8) \
M(T, UInt16) \
M(T, UInt32) \
M(T, UInt64) \
M(T, Int8) \
M(T, Int16) \
M(T, Int32) \
M(T, Int64) \
M(T, Float32) \
M(T, Float64)
#define FOR_LEASTSQR_TYPES(M) \
FOR_LEASTSQR_TYPES_2(M, UInt8) \
FOR_LEASTSQR_TYPES_2(M, UInt16) \
FOR_LEASTSQR_TYPES_2(M, UInt32) \
FOR_LEASTSQR_TYPES_2(M, UInt64) \
FOR_LEASTSQR_TYPES_2(M, Int8) \
FOR_LEASTSQR_TYPES_2(M, Int16) \
FOR_LEASTSQR_TYPES_2(M, Int32) \
FOR_LEASTSQR_TYPES_2(M, Int64) \
FOR_LEASTSQR_TYPES_2(M, Float32) \
FOR_LEASTSQR_TYPES_2(M, Float64)
#define DISPATCH(T1, T2) \
if (which_x.idx == TypeIndex::T1 && which_y.idx == TypeIndex::T2) \
return std::make_shared<AggregateFunctionLeastSqr<T1, T2>>( \
arguments, \
params \
);
FOR_LEASTSQR_TYPES(DISPATCH)
#undef FOR_LEASTSQR_TYPES_2
#undef FOR_LEASTSQR_TYPES
#undef DISPATCH
throw Exception(
"Illegal types ("
+ x_arg->getName() + ", " + y_arg->getName()
+ ") of arguments of aggregate function " + name
+ ", must be Native Ints, Native UInts or Floats",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
);
}
}
void registerAggregateFunctionLeastSqr(AggregateFunctionFactory & factory)
{
factory.registerFunction("leastSqr", createAggregateFunctionLeastSqr);
}
}

View File

@ -0,0 +1,195 @@
#pragma once
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <limits>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
template <typename X, typename Y, typename Ret>
struct AggregateFunctionLeastSqrData final
{
size_t count = 0;
Ret sum_x = 0;
Ret sum_y = 0;
Ret sum_xx = 0;
Ret sum_xy = 0;
void add(X x, Y y)
{
count += 1;
sum_x += x;
sum_y += y;
sum_xx += x * x;
sum_xy += x * y;
}
void merge(const AggregateFunctionLeastSqrData & other)
{
count += other.count;
sum_x += other.sum_x;
sum_y += other.sum_y;
sum_xx += other.sum_xx;
sum_xy += other.sum_xy;
}
void serialize(WriteBuffer & buf) const
{
writeBinary(count, buf);
writeBinary(sum_x, buf);
writeBinary(sum_y, buf);
writeBinary(sum_xx, buf);
writeBinary(sum_xy, buf);
}
void deserialize(ReadBuffer & buf)
{
readBinary(count, buf);
readBinary(sum_x, buf);
readBinary(sum_y, buf);
readBinary(sum_xx, buf);
readBinary(sum_xy, buf);
}
Ret getK() const
{
Ret divisor = sum_xx * count - sum_x * sum_x;
if (divisor == 0)
return std::numeric_limits<Ret>::quiet_NaN();
return (sum_xy * count - sum_x * sum_y) / divisor;
}
Ret getB(Ret k) const
{
if (count == 0)
return std::numeric_limits<Ret>::quiet_NaN();
return (sum_y - k * sum_x) / count;
}
};
/// Calculates simple linear regression parameters.
/// Result is a tuple (k, b) for y = k * x + b equation, solved by least squares approximation.
template <typename X, typename Y, typename Ret = Float64>
class AggregateFunctionLeastSqr final : public IAggregateFunctionDataHelper<
AggregateFunctionLeastSqrData<X, Y, Ret>,
AggregateFunctionLeastSqr<X, Y, Ret>
>
{
public:
AggregateFunctionLeastSqr(
const DataTypes & arguments,
const Array & params
):
IAggregateFunctionDataHelper<
AggregateFunctionLeastSqrData<X, Y, Ret>,
AggregateFunctionLeastSqr<X, Y, Ret>
> {arguments, params}
{
// notice: arguments has been checked before
}
String getName() const override
{
return "leastSqr";
}
const char * getHeaderFilePath() const override
{
return __FILE__;
}
void add(
AggregateDataPtr place,
const IColumn ** columns,
size_t row_num,
Arena *
) const override
{
auto col_x {
static_cast<const ColumnVector<X> *>(columns[0])
};
auto col_y {
static_cast<const ColumnVector<Y> *>(columns[1])
};
X x = col_x->getData()[row_num];
Y y = col_y->getData()[row_num];
this->data(place).add(x, y);
}
void merge(
AggregateDataPtr place,
ConstAggregateDataPtr rhs, Arena *
) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(
ConstAggregateDataPtr place,
WriteBuffer & buf
) const override
{
this->data(place).serialize(buf);
}
void deserialize(
AggregateDataPtr place,
ReadBuffer & buf, Arena *
) const override
{
this->data(place).deserialize(buf);
}
DataTypePtr getReturnType() const override
{
DataTypes types {
std::make_shared<DataTypeNumber<Ret>>(),
std::make_shared<DataTypeNumber<Ret>>(),
};
Strings names {
"k",
"b",
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
void insertResultInto(
ConstAggregateDataPtr place,
IColumn & to
) const override
{
Ret k = this->data(place).getK();
Ret b = this->data(place).getB(k);
auto & col_tuple = static_cast<ColumnTuple &>(to);
auto & col_k = static_cast<ColumnVector<Ret> &>(col_tuple.getColumn(0));
auto & col_b = static_cast<ColumnVector<Ret> &>(col_tuple.getColumn(1));
col_k.getData().push_back(k);
col_b.getData().push_back(b);
}
};
}

View File

@ -24,8 +24,7 @@ struct WithoutOverflowPolicy
static DataTypePtr promoteType(const DataTypePtr & data_type)
{
if (!data_type->canBePromoted())
throw new Exception{"Values to be summed are expected to be Numeric, Float or Decimal.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Values to be summed are expected to be Numeric, Float or Decimal.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return data_type->promoteNumericType();
}

View File

@ -16,7 +16,6 @@
#include <Common/HashTable/HashSet.h>
#include <Common/HyperLogLogWithSmallSetOptimization.h>
#include <Common/CombinedCardinalityEstimator.h>
#include <Common/MemoryTracker.h>
#include <Common/typeid_cast.h>
#include <AggregateFunctions/UniquesHashSet.h>

View File

@ -29,6 +29,7 @@ void registerAggregateFunctionsBitwise(AggregateFunctionFactory &);
void registerAggregateFunctionsBitmap(AggregateFunctionFactory &);
void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory &);
void registerAggregateFunctionEntropy(AggregateFunctionFactory &);
void registerAggregateFunctionLeastSqr(AggregateFunctionFactory &);
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorArray(AggregateFunctionCombinatorFactory &);
@ -69,6 +70,7 @@ void registerAggregateFunctions()
registerAggregateFunctionHistogram(factory);
registerAggregateFunctionRetention(factory);
registerAggregateFunctionEntropy(factory);
registerAggregateFunctionLeastSqr(factory);
}
{

View File

@ -271,7 +271,7 @@ private:
void initBlockInput();
void initBlockLogsInput();
void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const;
[[noreturn]] void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const;
};
}

View File

@ -10,7 +10,7 @@ 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.
* MemoryTracker is not used. AlignedBuffer is intended for small pieces of memory.
*/
class AlignedBuffer : private boost::noncopyable
{

View File

@ -1,190 +0,0 @@
#include <Common/Allocator.h>
#if !defined(__APPLE__) && !defined(__FreeBSD__)
#include <malloc.h>
#endif
#include <cstdlib>
#include <algorithm>
#include <sys/mman.h>
#include <Core/Defines.h>
#ifdef THREAD_SANITIZER
/// Thread sanitizer does not intercept mremap. The usage of mremap will lead to false positives.
#define DISABLE_MREMAP 1
#endif
#include <common/mremap.h>
#include <Common/MemoryTracker.h>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
#include <IO/WriteHelpers.h>
/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS
#ifndef MAP_ANONYMOUS
#define MAP_ANONYMOUS MAP_ANON
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int CANNOT_ALLOCATE_MEMORY;
extern const int CANNOT_MUNMAP;
extern const int CANNOT_MREMAP;
}
}
/** Many modern allocators (for example, tcmalloc) do not do a mremap for realloc,
* even in case of large enough chunks of memory.
* Although this allows you to increase performance and reduce memory consumption during realloc.
* To fix this, we do mremap manually if the chunk of memory is large enough.
* The threshold (64 MB) is chosen quite large, since changing the address space is
* very slow, especially in the case of a large number of threads.
* We expect that the set of operations mmap/something to do/mremap can only be performed about 1000 times per second.
*
* PS. This is also required, because tcmalloc can not allocate a chunk of memory greater than 16 GB.
*/
#ifdef NDEBUG
static constexpr size_t MMAP_THRESHOLD = 64 * (1ULL << 20);
#else
/// In debug build, use small mmap threshold to reproduce more memory stomping bugs.
/// Along with ASLR it will hopefully detect more issues than ASan.
/// The program may fail due to the limit on number of memory mappings.
static constexpr size_t MMAP_THRESHOLD = 4096;
#endif
static constexpr size_t MMAP_MIN_ALIGNMENT = 4096;
static constexpr size_t MALLOC_MIN_ALIGNMENT = 8;
template <bool clear_memory_>
void * Allocator<clear_memory_>::mmap_hint()
{
#if ALLOCATOR_ASLR
return reinterpret_cast<void *>(std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(rng));
#else
return nullptr;
#endif
}
template <bool clear_memory_>
void * Allocator<clear_memory_>::alloc(size_t size, size_t alignment)
{
CurrentMemoryTracker::alloc(size);
void * buf;
if (size >= MMAP_THRESHOLD)
{
if (alignment > MMAP_MIN_ALIGNMENT)
throw DB::Exception("Too large alignment " + formatReadableSizeWithBinarySuffix(alignment) + ": more than page size when allocating "
+ formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::BAD_ARGUMENTS);
buf = mmap(mmap_hint(), size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/// No need for zero-fill, because mmap guarantees it.
}
else
{
if (alignment <= MALLOC_MIN_ALIGNMENT)
{
if (clear_memory)
buf = ::calloc(size, 1);
else
buf = ::malloc(size);
if (nullptr == buf)
DB::throwFromErrno("Allocator: Cannot malloc " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
}
else
{
buf = nullptr;
int res = posix_memalign(&buf, alignment, size);
if (0 != res)
DB::throwFromErrno("Cannot allocate memory (posix_memalign) " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
if (clear_memory)
memset(buf, 0, size);
}
}
return buf;
}
template <bool clear_memory_>
void Allocator<clear_memory_>::free(void * buf, size_t size)
{
if (size >= MMAP_THRESHOLD)
{
if (0 != munmap(buf, size))
DB::throwFromErrno("Allocator: Cannot munmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_MUNMAP);
}
else
{
::free(buf);
}
CurrentMemoryTracker::free(size);
}
template <bool clear_memory_>
void * Allocator<clear_memory_>::realloc(void * buf, size_t old_size, size_t new_size, size_t alignment)
{
if (old_size == new_size)
{
/// nothing to do.
/// BTW, it's not possible to change alignment while doing realloc.
}
else if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD && alignment <= MALLOC_MIN_ALIGNMENT)
{
/// Resize malloc'd memory region with no special alignment requirement.
CurrentMemoryTracker::realloc(old_size, new_size);
void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf)
DB::throwFromErrno("Allocator: Cannot realloc from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
buf = new_buf;
if (clear_memory && new_size > old_size)
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
}
else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD)
{
/// Resize mmap'd memory region.
CurrentMemoryTracker::realloc(old_size, new_size);
// On apple and freebsd self-implemented mremap used (common/mremap.h)
buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mremap memory chunk from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_MREMAP);
/// No need for zero-fill, because mmap guarantees it.
}
else
{
/// All other cases that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods.
void * new_buf = alloc(new_size, alignment);
memcpy(new_buf, buf, std::min(old_size, new_size));
free(buf, old_size);
buf = new_buf;
}
return buf;
}
/// Explicit template instantiations.
template class Allocator<true>;
template class Allocator<false>;

View File

@ -10,11 +10,88 @@
#define ALLOCATOR_ASLR 1
#endif
#if ALLOCATOR_ASLR
#include <pcg_random.hpp>
#include <Common/randomSeed.h>
#if !defined(__APPLE__) && !defined(__FreeBSD__)
#include <malloc.h>
#endif
#include <cstdlib>
#include <algorithm>
#include <sys/mman.h>
#include <Core/Defines.h>
#ifdef THREAD_SANITIZER
/// Thread sanitizer does not intercept mremap. The usage of mremap will lead to false positives.
#define DISABLE_MREMAP 1
#endif
#include <common/mremap.h>
#include <Common/MemoryTracker.h>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS
#ifndef MAP_ANONYMOUS
#define MAP_ANONYMOUS MAP_ANON
#endif
/** Many modern allocators (for example, tcmalloc) do not do a mremap for realloc,
* even in case of large enough chunks of memory.
* Although this allows you to increase performance and reduce memory consumption during realloc.
* To fix this, we do mremap manually if the chunk of memory is large enough.
* The threshold (64 MB) is chosen quite large, since changing the address space is
* very slow, especially in the case of a large number of threads.
* We expect that the set of operations mmap/something to do/mremap can only be performed about 1000 times per second.
*
* PS. This is also required, because tcmalloc can not allocate a chunk of memory greater than 16 GB.
*/
#ifdef NDEBUG
static constexpr size_t MMAP_THRESHOLD = 64 * (1ULL << 20);
#else
/// In debug build, use small mmap threshold to reproduce more memory stomping bugs.
/// Along with ASLR it will hopefully detect more issues than ASan.
/// The program may fail due to the limit on number of memory mappings.
static constexpr size_t MMAP_THRESHOLD = 4096;
#endif
static constexpr size_t MMAP_MIN_ALIGNMENT = 4096;
static constexpr size_t MALLOC_MIN_ALIGNMENT = 8;
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int CANNOT_ALLOCATE_MEMORY;
extern const int CANNOT_MUNMAP;
extern const int CANNOT_MREMAP;
}
}
namespace AllocatorHints
{
struct DefaultHint
{
void * mmap_hint()
{
return nullptr;
}
};
struct RandomHint
{
void * mmap_hint()
{
return reinterpret_cast<void *>(std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(rng));
}
private:
pcg64 rng{randomSeed()};
};
}
/** Responsible for allocating / freeing memory. Used, for example, in PODArray, Arena.
* Also used in hash tables.
@ -23,31 +100,126 @@
* - passing the size into the `free` method;
* - by the presence of the `alignment` argument;
* - the possibility of zeroing memory (used in hash tables);
* - hint class for mmap
* - mmap_threshold for using mmap less or more
*/
template <bool clear_memory_>
class Allocator
template <bool clear_memory_, typename Hint, size_t mmap_threshold>
class AllocatorWithHint : Hint
{
#if ALLOCATOR_ASLR
private:
pcg64 rng{randomSeed()};
#endif
void * mmap_hint();
protected:
static constexpr bool clear_memory = clear_memory_;
public:
/// Allocate memory range.
void * alloc(size_t size, size_t alignment = 0);
void * alloc(size_t size, size_t alignment = 0)
{
CurrentMemoryTracker::alloc(size);
void * buf;
if (size >= mmap_threshold)
{
if (alignment > MMAP_MIN_ALIGNMENT)
throw DB::Exception("Too large alignment " + formatReadableSizeWithBinarySuffix(alignment) + ": more than page size when allocating "
+ formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::BAD_ARGUMENTS);
buf = mmap(Hint::mmap_hint(), size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/// No need for zero-fill, because mmap guarantees it.
}
else
{
if (alignment <= MALLOC_MIN_ALIGNMENT)
{
if constexpr (clear_memory)
buf = ::calloc(size, 1);
else
buf = ::malloc(size);
if (nullptr == buf)
DB::throwFromErrno("Allocator: Cannot malloc " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
}
else
{
buf = nullptr;
int res = posix_memalign(&buf, alignment, size);
if (0 != res)
DB::throwFromErrno("Cannot allocate memory (posix_memalign) " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
if (clear_memory)
memset(buf, 0, size);
}
}
return buf;
}
/// Free memory range.
void free(void * buf, size_t size);
void free(void * buf, size_t size)
{
if (size >= mmap_threshold)
{
if (0 != munmap(buf, size))
DB::throwFromErrno("Allocator: Cannot munmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_MUNMAP);
}
else
{
::free(buf);
}
CurrentMemoryTracker::free(size);
}
/** Enlarge memory range.
* Data from old range is moved to the beginning of new range.
* Address of memory range could change.
*/
void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0);
void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0)
{
if (old_size == new_size)
{
/// nothing to do.
/// BTW, it's not possible to change alignment while doing realloc.
}
else if (old_size < mmap_threshold && new_size < mmap_threshold && alignment <= MALLOC_MIN_ALIGNMENT)
{
/// Resize malloc'd memory region with no special alignment requirement.
CurrentMemoryTracker::realloc(old_size, new_size);
void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf)
DB::throwFromErrno("Allocator: Cannot realloc from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
buf = new_buf;
if (clear_memory && new_size > old_size)
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
}
else if (old_size >= mmap_threshold && new_size >= mmap_threshold)
{
/// Resize mmap'd memory region.
CurrentMemoryTracker::realloc(old_size, new_size);
// On apple and freebsd self-implemented mremap used (common/mremap.h)
buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mremap memory chunk from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_MREMAP);
/// No need for zero-fill, because mmap guarantees it.
}
else
{
/// All other cases that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods.
void * new_buf = alloc(new_size, alignment);
memcpy(new_buf, buf, std::min(old_size, new_size));
free(buf, old_size);
buf = new_buf;
}
return buf;
}
protected:
static constexpr size_t getStackThreshold()
@ -56,6 +228,13 @@ protected:
}
};
#if ALLOCATOR_ASLR
template <bool clear_memory>
using Allocator = AllocatorWithHint<clear_memory, AllocatorHints::RandomHint, MMAP_THRESHOLD>;
#else
template <bool clear_memory>
using Allocator = AllocatorWithHint<clear_memory, AllocatorHints::DefaultHint, MMAP_THRESHOLD>;
#endif
/** When using AllocatorWithStackMemory, located on the stack,
* GCC 4.9 mistakenly assumes that we can call `free` from a pointer to the stack.

View File

@ -49,7 +49,7 @@ private:
ProfileEvents::increment(ProfileEvents::ArenaAllocChunks);
ProfileEvents::increment(ProfileEvents::ArenaAllocBytes, size_);
begin = reinterpret_cast<char *>(Allocator::alloc(size_));
begin = reinterpret_cast<char *>(Allocator<false>::alloc(size_));
pos = begin;
end = begin + size_ - pad_right;
prev = prev_;
@ -57,7 +57,7 @@ private:
~Chunk()
{
Allocator::free(begin, size());
Allocator<false>::free(begin, size());
if (prev)
delete prev;

View File

@ -55,7 +55,7 @@ public:
char * alloc(const size_t size)
{
if (size > max_fixed_block_size)
return static_cast<char *>(Allocator::alloc(size));
return static_cast<char *>(Allocator<false>::alloc(size));
/// find list of required size
const auto list_idx = findFreeListIndex(size);
@ -76,7 +76,7 @@ public:
void free(char * ptr, const size_t size)
{
if (size > max_fixed_block_size)
return Allocator::free(ptr, size);
return Allocator<false>::free(ptr, size);
/// find list of required size
const auto list_idx = findFreeListIndex(size);

View File

@ -421,6 +421,9 @@ namespace ErrorCodes
extern const int UNKNOWN_PROTOBUF_FORMAT = 444;
extern const int CANNOT_MPROTECT = 445;
extern const int FUNCTION_NOT_ALLOWED = 446;
extern const int HYPERSCAN_CANNOT_SCAN_TEXT = 447;
extern const int BROTLI_READ_FAILED = 448;
extern const int BROTLI_WRITE_FAILED = 449;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -21,11 +21,6 @@ namespace ErrorCodes
extern const int CANNOT_TRUNCATE_FILE;
}
const char * getVersion()
{
return VERSION_STRING;
}
std::string errnoToString(int code, int e)
{
const size_t buf_size = 128;
@ -82,14 +77,15 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
}
catch (const Exception & e)
{
stream << "(version " << getVersion() << ") " << getExceptionMessage(e, with_stacktrace, check_embedded_stacktrace);
stream << getExceptionMessage(e, with_stacktrace, check_embedded_stacktrace) << " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
}
catch (const Poco::Exception & e)
{
try
{
stream << "(version " << getVersion() << ") " << "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText();
stream << "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText()
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
}
catch (...) {}
}
@ -103,7 +99,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
if (status)
name += " (demangling status: " + toString(status) + ")";
stream << "(version " << getVersion() << ") " << "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what();
stream << "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what() << ", version = " << VERSION_STRING << VERSION_OFFICIAL;
}
catch (...) {}
}
@ -117,7 +113,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
if (status)
name += " (demangling status: " + toString(status) + ")";
stream << "(version " << getVersion() << ") " << "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name;
stream << "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name << " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
}
catch (...) {}
}

View File

@ -20,7 +20,7 @@ namespace ErrorCodes
* template parameter is available as Creator
*/
template <typename CreatorFunc>
class IFactoryWithAliases
class IFactoryWithAliases : public IHints<2, IFactoryWithAliases<CreatorFunc>>
{
protected:
using Creator = CreatorFunc;
@ -76,7 +76,7 @@ public:
throw Exception(factory_name + ": alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
}
std::vector<String> getAllRegisteredNames() const
std::vector<String> getAllRegisteredNames() const override
{
std::vector<String> result;
auto getter = [](const auto & pair) { return pair.first; };
@ -106,13 +106,7 @@ public:
return aliases.count(name) || case_insensitive_aliases.count(name);
}
std::vector<String> getHints(const String & name) const
{
static const auto registered_names = getAllRegisteredNames();
return prompter.getHints(name, registered_names);
}
virtual ~IFactoryWithAliases() {}
virtual ~IFactoryWithAliases() override {}
private:
using InnerMap = std::unordered_map<String, Creator>; // name -> creator
@ -127,13 +121,6 @@ private:
/// Case insensitive aliases
AliasMap case_insensitive_aliases;
/**
* prompter for names, if a person makes a typo for some function or type, it
* helps to find best possible match (in particular, edit distance is done like in clang
* (max edit distance is (typo.size() + 2) / 3)
*/
NamePrompter</*MaxNumHints=*/2> prompter;
};
}

View File

@ -97,4 +97,23 @@ private:
}
};
template <size_t MaxNumHints, class Self>
class IHints
{
public:
virtual std::vector<String> getAllRegisteredNames() const = 0;
std::vector<String> getHints(const String & name) const
{
static const auto registered_names = getAllRegisteredNames();
return prompter.getHints(name, registered_names);
}
virtual ~IHints() = default;
private:
NamePrompter<MaxNumHints> prompter;
};
}

View File

@ -156,7 +156,7 @@ public:
#endif
}
bool compare(const UInt8 * pos) const
ALWAYS_INLINE bool compare(const UInt8 * pos) const
{
static const Poco::UTF8Encoding utf8;
@ -374,7 +374,7 @@ public:
#endif
}
bool compare(const UInt8 * pos) const
ALWAYS_INLINE bool compare(const UInt8 * pos) const
{
#ifdef __SSE4_1__
if (pageSafe(pos))
@ -568,7 +568,7 @@ public:
#endif
}
bool compare(const UInt8 * pos) const
ALWAYS_INLINE bool compare(const UInt8 * pos) const
{
#ifdef __SSE4_1__
if (pageSafe(pos))

View File

@ -20,6 +20,7 @@
#cmakedefine VERSION_MINOR @VERSION_MINOR@
#cmakedefine VERSION_PATCH @VERSION_PATCH@
#cmakedefine VERSION_STRING "@VERSION_STRING@"
#cmakedefine VERSION_OFFICIAL "@VERSION_OFFICIAL@"
#cmakedefine VERSION_FULL "@VERSION_FULL@"
#cmakedefine VERSION_DESCRIBE "@VERSION_DESCRIBE@"
#cmakedefine VERSION_GITHASH "@VERSION_GITHASH@"
@ -42,3 +43,7 @@
#else
#define DBMS_VERSION_PATCH 0
#endif
#if !defined(VERSION_OFFICIAL)
# define VERSION_OFFICIAL ""
#endif

View File

@ -35,7 +35,7 @@ bool CachedCompressedReadBuffer::nextImpl()
UInt128 key = cache->hash(path, file_pos);
owned_cell = cache->get(key);
if (!owned_cell || !codec)
if (!owned_cell)
{
/// If not, read it from the file.
initInput();
@ -49,21 +49,22 @@ bool CachedCompressedReadBuffer::nextImpl()
if (owned_cell->compressed_size)
{
owned_cell->data.resize(size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer());
owned_cell->additional_bytes = codec->getAdditionalSizeAtTheEndOfBuffer();
owned_cell->data.resize(size_decompressed + owned_cell->additional_bytes);
decompress(owned_cell->data.data(), size_decompressed, size_compressed_without_checksum);
/// Put data into cache.
cache->set(key, owned_cell);
}
/// Put data into cache.
/// NOTE: Even if we don't read anything (compressed_size == 0)
/// because we can reuse this information and don't reopen file in future
cache->set(key, owned_cell);
}
if (owned_cell->data.size() == 0)
{
owned_cell = nullptr;
return false;
}
working_buffer = Buffer(owned_cell->data.data(), owned_cell->data.data() + owned_cell->data.size() - codec->getAdditionalSizeAtTheEndOfBuffer());
working_buffer = Buffer(owned_cell->data.data(), owned_cell->data.data() + owned_cell->data.size() - owned_cell->additional_bytes);
file_pos += owned_cell->compressed_size;

View File

@ -125,10 +125,10 @@ void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_
}
}
void registerCodecDelta(CompressionCodecFactory & factory)
namespace
{
UInt8 method_code = UInt8(CompressionMethodByte::Delta);
factory.registerCompressionCodecWithType("Delta", method_code, [&](const ASTPtr & arguments, DataTypePtr column_type) -> CompressionCodecPtr
UInt8 getDeltaBytesSize(DataTypePtr column_type)
{
UInt8 delta_bytes_size = 1;
if (column_type && column_type->haveMaximumSizeOfValue())
@ -137,7 +137,22 @@ void registerCodecDelta(CompressionCodecFactory & factory)
if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8)
delta_bytes_size = static_cast<UInt8>(max_size);
}
return delta_bytes_size;
}
}
void CompressionCodecDelta::useInfoAboutType(DataTypePtr data_type)
{
delta_bytes_size = getDeltaBytesSize(data_type);
}
void registerCodecDelta(CompressionCodecFactory & factory)
{
UInt8 method_code = UInt8(CompressionMethodByte::Delta);
factory.registerCompressionCodecWithType("Delta", method_code, [&](const ASTPtr & arguments, DataTypePtr column_type) -> CompressionCodecPtr
{
UInt8 delta_bytes_size = getDeltaBytesSize(column_type);
if (arguments && !arguments->children.empty())
{
if (arguments->children.size() > 1)

View File

@ -1,6 +1,7 @@
#pragma once
#include <Compression/ICompressionCodec.h>
namespace DB
{
@ -13,14 +14,18 @@ public:
String getCodecDesc() const override;
void useInfoAboutType(DataTypePtr data_type) override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override { return uncompressed_size + 2; }
private:
const UInt8 delta_bytes_size;
};
}
private:
UInt8 delta_bytes_size;
};
}

View File

@ -21,16 +21,6 @@ extern const int CORRUPTED_DATA;
CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs)
: codecs(codecs)
{
std::ostringstream ss;
for (size_t idx = 0; idx < codecs.size(); idx++)
{
if (idx != 0)
ss << ',' << ' ';
const auto codec = codecs[idx];
ss << codec->getCodecDesc();
}
codec_desc = ss.str();
}
UInt8 CompressionCodecMultiple::getMethodByte() const
@ -40,7 +30,16 @@ UInt8 CompressionCodecMultiple::getMethodByte() const
String CompressionCodecMultiple::getCodecDesc() const
{
return codec_desc;
std::ostringstream ss;
for (size_t idx = 0; idx < codecs.size(); idx++)
{
if (idx != 0)
ss << ',' << ' ';
const auto codec = codecs[idx];
ss << codec->getCodecDesc();
}
return ss.str();
}
UInt32 CompressionCodecMultiple::getMaxCompressedDataSize(UInt32 uncompressed_size) const
@ -79,6 +78,14 @@ UInt32 CompressionCodecMultiple::doCompressData(const char * source, UInt32 sour
return 1 + codecs.size() + source_size;
}
void CompressionCodecMultiple::useInfoAboutType(DataTypePtr data_type)
{
for (auto & codec : codecs)
{
codec->useInfoAboutType(data_type);
}
}
void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 decompressed_size) const
{
UInt8 compression_methods_size = source[0];

View File

@ -17,6 +17,8 @@ public:
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
void useInfoAboutType(DataTypePtr data_type) override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
@ -24,7 +26,6 @@ protected:
private:
Codecs codecs;
String codec_desc;
};

View File

@ -42,17 +42,6 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std
}
}
CompressionCodecPtr CompressionCodecFactory::get(const std::vector<CodecNameWithLevel> & codecs) const
{
Codecs result;
for (const auto & [codec_name, level] : codecs)
result.push_back(get(codec_name, level));
if (result.size() == 1)
return result.back();
return std::make_shared<CompressionCodecMultiple>(result);
}
CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, DataTypePtr column_type) const
{
@ -93,7 +82,7 @@ CompressionCodecPtr CompressionCodecFactory::get(const UInt8 byte_code) const
CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, const ASTPtr & arguments, DataTypePtr column_type) const
{
if (family_name == "Multiple")
throw Exception("Codec MULTIPLE cannot be specified directly", ErrorCodes::UNKNOWN_CODEC);
throw Exception("Codec Multiple cannot be specified directly", ErrorCodes::UNKNOWN_CODEC);
const auto family_and_creator = family_name_with_codec.find(family_name);

View File

@ -48,8 +48,6 @@ public:
/// For backward compatibility with config settings
CompressionCodecPtr get(const String & family_name, std::optional<int> level) const;
CompressionCodecPtr get(const std::vector<CodecNameWithLevel> & codecs) const;
/// Register codec with parameters and column type
void registerCompressionCodecWithType(const String & family_name, std::optional<UInt8> byte_code, CreatorWithType creator);
/// Register codec with parameters

View File

@ -58,6 +58,9 @@ public:
/// Read method byte from compressed source
static UInt8 readMethod(const char * source);
/// Some codecs may use information about column type which appears after codec creation
virtual void useInfoAboutType(DataTypePtr /* data_type */) { }
protected:
/// Return size of compressed data without header

View File

@ -23,20 +23,21 @@ namespace DB
class TaskNotification final : public Poco::Notification
{
public:
explicit TaskNotification(const BackgroundSchedulePool::TaskInfoPtr & task) : task(task) {}
explicit TaskNotification(const BackgroundSchedulePoolTaskInfoPtr & task) : task(task) {}
void execute() { task->execute(); }
private:
BackgroundSchedulePool::TaskInfoPtr task;
BackgroundSchedulePoolTaskInfoPtr task;
};
BackgroundSchedulePool::TaskInfo::TaskInfo(BackgroundSchedulePool & pool_, const std::string & log_name_, const TaskFunc & function_)
BackgroundSchedulePoolTaskInfo::BackgroundSchedulePoolTaskInfo(
BackgroundSchedulePool & pool_, const std::string & log_name_, const BackgroundSchedulePool::TaskFunc & function_)
: pool(pool_), log_name(log_name_), function(function_)
{
}
bool BackgroundSchedulePool::TaskInfo::schedule()
bool BackgroundSchedulePoolTaskInfo::schedule()
{
std::lock_guard lock(schedule_mutex);
@ -47,7 +48,7 @@ bool BackgroundSchedulePool::TaskInfo::schedule()
return true;
}
bool BackgroundSchedulePool::TaskInfo::scheduleAfter(size_t ms)
bool BackgroundSchedulePoolTaskInfo::scheduleAfter(size_t ms)
{
std::lock_guard lock(schedule_mutex);
@ -58,7 +59,7 @@ bool BackgroundSchedulePool::TaskInfo::scheduleAfter(size_t ms)
return true;
}
void BackgroundSchedulePool::TaskInfo::deactivate()
void BackgroundSchedulePoolTaskInfo::deactivate()
{
std::lock_guard lock_exec(exec_mutex);
std::lock_guard lock_schedule(schedule_mutex);
@ -73,13 +74,13 @@ void BackgroundSchedulePool::TaskInfo::deactivate()
pool.cancelDelayedTask(shared_from_this(), lock_schedule);
}
void BackgroundSchedulePool::TaskInfo::activate()
void BackgroundSchedulePoolTaskInfo::activate()
{
std::lock_guard lock(schedule_mutex);
deactivated = false;
}
bool BackgroundSchedulePool::TaskInfo::activateAndSchedule()
bool BackgroundSchedulePoolTaskInfo::activateAndSchedule()
{
std::lock_guard lock(schedule_mutex);
@ -91,7 +92,7 @@ bool BackgroundSchedulePool::TaskInfo::activateAndSchedule()
return true;
}
void BackgroundSchedulePool::TaskInfo::execute()
void BackgroundSchedulePoolTaskInfo::execute()
{
Stopwatch watch;
CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundSchedulePoolTask};
@ -131,7 +132,7 @@ void BackgroundSchedulePool::TaskInfo::execute()
}
}
void BackgroundSchedulePool::TaskInfo::scheduleImpl(std::lock_guard<std::mutex> & schedule_mutex_lock)
void BackgroundSchedulePoolTaskInfo::scheduleImpl(std::lock_guard<std::mutex> & schedule_mutex_lock)
{
scheduled = true;
@ -145,7 +146,7 @@ void BackgroundSchedulePool::TaskInfo::scheduleImpl(std::lock_guard<std::mutex>
pool.queue.enqueueNotification(new TaskNotification(shared_from_this()));
}
Coordination::WatchCallback BackgroundSchedulePool::TaskInfo::getWatchCallback()
Coordination::WatchCallback BackgroundSchedulePoolTaskInfo::getWatchCallback()
{
return [t = shared_from_this()](const Coordination::WatchResponse &)
{

View File

@ -20,6 +20,8 @@ namespace DB
{
class TaskNotification;
class BackgroundSchedulePoolTaskInfo;
class BackgroundSchedulePoolTaskHolder;
/** Executes functions scheduled at a specific point in time.
@ -35,84 +37,14 @@ class TaskNotification;
class BackgroundSchedulePool
{
public:
class TaskInfo;
friend class BackgroundSchedulePoolTaskInfo;
using TaskInfo = BackgroundSchedulePoolTaskInfo;
using TaskInfoPtr = std::shared_ptr<TaskInfo>;
using TaskFunc = std::function<void()>;
using TaskHolder = BackgroundSchedulePoolTaskHolder;
using DelayedTasks = std::multimap<Poco::Timestamp, TaskInfoPtr>;
class TaskInfo : public std::enable_shared_from_this<TaskInfo>, private boost::noncopyable
{
public:
TaskInfo(BackgroundSchedulePool & pool_, const std::string & log_name_, const TaskFunc & function_);
/// Schedule for execution as soon as possible (if not already scheduled).
/// If the task was already scheduled with delay, the delay will be ignored.
bool schedule();
/// Schedule for execution after specified delay.
bool scheduleAfter(size_t ms);
/// Further attempts to schedule become no-op. Will wait till the end of the current execution of the task.
void deactivate();
void activate();
/// Atomically activate task and schedule it for execution.
bool activateAndSchedule();
/// get Coordination::WatchCallback needed for notifications from ZooKeeper watches.
Coordination::WatchCallback getWatchCallback();
private:
friend class TaskNotification;
friend class BackgroundSchedulePool;
void execute();
void scheduleImpl(std::lock_guard<std::mutex> & schedule_mutex_lock);
BackgroundSchedulePool & pool;
std::string log_name;
TaskFunc function;
std::mutex exec_mutex;
std::mutex schedule_mutex;
/// Invariants:
/// * If deactivated is true then scheduled, delayed and executing are all false.
/// * scheduled and delayed cannot be true at the same time.
bool deactivated = false;
bool scheduled = false;
bool delayed = false;
bool executing = false;
/// If the task is scheduled with delay, points to element of delayed_tasks.
DelayedTasks::iterator iterator;
};
class TaskHolder
{
public:
TaskHolder() = default;
explicit TaskHolder(const TaskInfoPtr & task_info_) : task_info(task_info_) {}
TaskHolder(const TaskHolder & other) = delete;
TaskHolder(TaskHolder && other) noexcept = default;
TaskHolder & operator=(const TaskHolder & other) noexcept = delete;
TaskHolder & operator=(TaskHolder && other) noexcept = default;
~TaskHolder()
{
if (task_info)
task_info->deactivate();
}
TaskInfo * operator->() { return task_info.get(); }
const TaskInfo * operator->() const { return task_info.get(); }
private:
TaskInfoPtr task_info;
};
TaskHolder createTask(const std::string & log_name, const TaskFunc & function);
size_t getNumberOfThreads() const { return size; }
@ -153,4 +85,81 @@ private:
void attachToThreadGroup();
};
class BackgroundSchedulePoolTaskInfo : public std::enable_shared_from_this<BackgroundSchedulePoolTaskInfo>, private boost::noncopyable
{
public:
BackgroundSchedulePoolTaskInfo(BackgroundSchedulePool & pool_, const std::string & log_name_, const BackgroundSchedulePool::TaskFunc & function_);
/// Schedule for execution as soon as possible (if not already scheduled).
/// If the task was already scheduled with delay, the delay will be ignored.
bool schedule();
/// Schedule for execution after specified delay.
bool scheduleAfter(size_t ms);
/// Further attempts to schedule become no-op. Will wait till the end of the current execution of the task.
void deactivate();
void activate();
/// Atomically activate task and schedule it for execution.
bool activateAndSchedule();
/// get Coordination::WatchCallback needed for notifications from ZooKeeper watches.
Coordination::WatchCallback getWatchCallback();
private:
friend class TaskNotification;
friend class BackgroundSchedulePool;
void execute();
void scheduleImpl(std::lock_guard<std::mutex> & schedule_mutex_lock);
BackgroundSchedulePool & pool;
std::string log_name;
BackgroundSchedulePool::TaskFunc function;
std::mutex exec_mutex;
std::mutex schedule_mutex;
/// Invariants:
/// * If deactivated is true then scheduled, delayed and executing are all false.
/// * scheduled and delayed cannot be true at the same time.
bool deactivated = false;
bool scheduled = false;
bool delayed = false;
bool executing = false;
/// If the task is scheduled with delay, points to element of delayed_tasks.
BackgroundSchedulePool::DelayedTasks::iterator iterator;
};
using BackgroundSchedulePoolTaskInfoPtr = std::shared_ptr<BackgroundSchedulePoolTaskInfo>;
class BackgroundSchedulePoolTaskHolder
{
public:
BackgroundSchedulePoolTaskHolder() = default;
explicit BackgroundSchedulePoolTaskHolder(const BackgroundSchedulePoolTaskInfoPtr & task_info_) : task_info(task_info_) {}
BackgroundSchedulePoolTaskHolder(const BackgroundSchedulePoolTaskHolder & other) = delete;
BackgroundSchedulePoolTaskHolder(BackgroundSchedulePoolTaskHolder && other) noexcept = default;
BackgroundSchedulePoolTaskHolder & operator=(const BackgroundSchedulePoolTaskHolder & other) noexcept = delete;
BackgroundSchedulePoolTaskHolder & operator=(BackgroundSchedulePoolTaskHolder && other) noexcept = default;
~BackgroundSchedulePoolTaskHolder()
{
if (task_info)
task_info->deactivate();
}
BackgroundSchedulePoolTaskInfo * operator->() { return task_info.get(); }
const BackgroundSchedulePoolTaskInfo * operator->() const { return task_info.get(); }
private:
BackgroundSchedulePoolTaskInfoPtr task_info;
};
}

View File

@ -533,12 +533,6 @@ void SettingString::write(WriteBuffer & buf) const
}
void SettingChar::checkStringIsACharacter(const String & x) const
{
if (x.size() != 1)
throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH);
}
String SettingChar::toString() const
{
return String(1, value);
@ -552,9 +546,10 @@ void SettingChar::set(char x)
void SettingChar::set(const String & x)
{
checkStringIsACharacter(x);
value = x[0];
changed = true;
if (x.size() > 1)
throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH);
char c = (x.size() == 1) ? x[0] : '\0';
set(c);
}
void SettingChar::set(const Field & x)
@ -565,10 +560,9 @@ void SettingChar::set(const Field & x)
void SettingChar::set(ReadBuffer & buf)
{
String x;
readBinary(x, buf);
checkStringIsACharacter(x);
set(x);
String s;
readBinary(s, buf);
set(s);
}
void SettingChar::write(WriteBuffer & buf) const

View File

@ -335,9 +335,6 @@ struct SettingString
struct SettingChar
{
private:
void checkStringIsACharacter(const String & x) const;
public:
char value;
bool changed = false;

View File

@ -5,8 +5,6 @@
#include <DataStreams/IBlockInputStream.h>
#include <Common/CurrentMetrics.h>
#include <Common/ThreadPool.h>
#include <Common/MemoryTracker.h>
#include <Poco/Ext/ThreadNumber.h>
namespace CurrentMetrics

View File

@ -43,6 +43,9 @@ struct BlockIO
BlockIO & operator= (const BlockIO & rhs)
{
if (this == &rhs)
return *this;
out.reset();
in.reset();
process_list_entry.reset();

View File

@ -1,7 +1,6 @@
#include <future>
#include <Common/setThreadName.h>
#include <Common/CurrentMetrics.h>
#include <Common/MemoryTracker.h>
#include <DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h>
#include <Common/CurrentThread.h>

View File

@ -8,8 +8,6 @@
#include <condition_variable>
class MemoryTracker;
namespace DB
{

View File

@ -11,7 +11,6 @@
#include <DataStreams/IBlockInputStream.h>
#include <Common/setThreadName.h>
#include <Common/CurrentMetrics.h>
#include <Common/MemoryTracker.h>
#include <Common/CurrentThread.h>
#include <Common/ThreadPool.h>

View File

@ -20,7 +20,7 @@ namespace ErrorCodes
namespace
{
class DataTypeDomanIPv4 : public DataTypeDomainWithSimpleSerialization
class DataTypeDomainIPv4 : public DataTypeDomainWithSimpleSerialization
{
public:
const char * getName() const override
@ -63,7 +63,7 @@ public:
}
};
class DataTypeDomanIPv6 : public DataTypeDomainWithSimpleSerialization
class DataTypeDomainIPv6 : public DataTypeDomainWithSimpleSerialization
{
public:
const char * getName() const override
@ -111,8 +111,8 @@ public:
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
{
factory.registerDataTypeDomain("UInt32", std::make_unique<DataTypeDomanIPv4>());
factory.registerDataTypeDomain("FixedString(16)", std::make_unique<DataTypeDomanIPv6>());
factory.registerDataTypeDomain("UInt32", std::make_unique<DataTypeDomainIPv4>());
factory.registerDataTypeDomain("FixedString(16)", std::make_unique<DataTypeDomainIPv6>());
}
} // namespace DB

View File

@ -690,10 +690,9 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams(
};
if (!settings.continuous_reading)
{
low_cardinality_state->num_pending_rows = 0;
if (!settings.continuous_reading)
{
/// Remember in state that some granules were skipped and we need to update dictionary.
low_cardinality_state->need_update_dictionary = true;
}

View File

@ -14,7 +14,7 @@ namespace DB
class IDataTypeDummy : public DataTypeWithSimpleSerialization
{
private:
void throwNoSerialization() const
[[noreturn]] void throwNoSerialization() const
{
throw Exception("Serialization is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -305,7 +305,7 @@ void registerInputFormatCapnProto(FormatFactory & factory)
[](ReadBuffer & buf, const Block & sample, const Context & context, UInt64 max_block_size, const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<CapnProtoRowInputStream>(buf, sample, FormatSchemaInfo(context, "capnp")),
std::make_shared<CapnProtoRowInputStream>(buf, sample, FormatSchemaInfo(context, "CapnProto")),
sample,
max_block_size,
settings);

View File

@ -11,20 +11,29 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & schema_file_extension, bool schema_required)
namespace
{
String getFormatSchemaDefaultFileExtension(const String & format)
{
if (format == "Protobuf")
return "proto";
else if (format == "CapnProto")
return "capnp";
else
return "";
}
}
FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & format)
{
String format_schema = context.getSettingsRef().format_schema.toString();
if (format_schema.empty())
{
if (schema_required)
{
throw Exception(
"Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format"
+ (schema_file_extension.empty() ? "" : ", e.g. 'schema." + schema_file_extension + ":Message'"),
ErrorCodes::BAD_ARGUMENTS);
}
return;
}
"The format " + format + " requires a schema. The 'format_schema' setting should be set", ErrorCodes::BAD_ARGUMENTS);
String default_file_extension = getFormatSchemaDefaultFileExtension(format);
size_t colon_pos = format_schema.find(':');
Poco::Path path;
@ -33,12 +42,11 @@ FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & schem
{
throw Exception(
"Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format"
+ (schema_file_extension.empty() ? "" : ", e.g. 'schema." + schema_file_extension + ":Message'") + ". Got '" + format_schema
+ (default_file_extension.empty() ? "" : ", e.g. 'schema." + default_file_extension + ":Message'") + ". Got '" + format_schema
+ "'",
ErrorCodes::BAD_ARGUMENTS);
}
is_null = false;
message_name = format_schema.substr(colon_pos + 1);
auto default_schema_directory = [&context]()
@ -51,8 +59,8 @@ FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & schem
return context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER);
};
if (path.getExtension().empty() && !schema_file_extension.empty())
path.setExtension(schema_file_extension);
if (path.getExtension().empty() && !default_file_extension.empty())
path.setExtension(default_file_extension);
if (path.isAbsolute())
{

View File

@ -10,10 +10,7 @@ class Context;
class FormatSchemaInfo
{
public:
FormatSchemaInfo() = default;
FormatSchemaInfo(const Context & context, const String & schema_file_extension = String(), bool schema_required = true);
bool isNull() const { return is_null; }
FormatSchemaInfo(const Context & context, const String & format);
/// Returns path to the schema file.
const String & schemaPath() const { return schema_path; }
@ -26,7 +23,6 @@ public:
const String & messageName() const { return message_name; }
private:
bool is_null = true;
String schema_path;
String schema_directory;
String message_name;

View File

@ -77,7 +77,7 @@ namespace ProtobufColumnMatcher
namespace details
{
void throwNoCommonColumns();
[[noreturn]] void throwNoCommonColumns();
class ColumnNameMatcher
{

View File

@ -385,73 +385,61 @@ public:
bool readStringInto(PaddedPODArray<UInt8> &) override
{
cannotConvertType("String");
return false;
}
bool readInt8(Int8 &) override
{
cannotConvertType("Int8");
return false;
}
bool readUInt8(UInt8 &) override
{
cannotConvertType("UInt8");
return false;
}
bool readInt16(Int16 &) override
{
cannotConvertType("Int16");
return false;
}
bool readUInt16(UInt16 &) override
{
cannotConvertType("UInt16");
return false;
}
bool readInt32(Int32 &) override
{
cannotConvertType("Int32");
return false;
}
bool readUInt32(UInt32 &) override
{
cannotConvertType("UInt32");
return false;
}
bool readInt64(Int64 &) override
{
cannotConvertType("Int64");
return false;
}
bool readUInt64(UInt64 &) override
{
cannotConvertType("UInt64");
return false;
}
bool readUInt128(UInt128 &) override
{
cannotConvertType("UInt128");
return false;
}
bool readFloat32(Float32 &) override
{
cannotConvertType("Float32");
return false;
}
bool readFloat64(Float64 &) override
{
cannotConvertType("Float64");
return false;
}
void prepareEnumMapping8(const std::vector<std::pair<std::string, Int8>> &) override {}
@ -460,59 +448,50 @@ public:
bool readEnum8(Int8 &) override
{
cannotConvertType("Enum");
return false;
}
bool readEnum16(Int16 &) override
{
cannotConvertType("Enum");
return false;
}
bool readUUID(UUID &) override
{
cannotConvertType("UUID");
return false;
}
bool readDate(DayNum &) override
{
cannotConvertType("Date");
return false;
}
bool readDateTime(time_t &) override
{
cannotConvertType("DateTime");
return false;
}
bool readDecimal32(Decimal32 &, UInt32, UInt32) override
{
cannotConvertType("Decimal32");
return false;
}
bool readDecimal64(Decimal64 &, UInt32, UInt32) override
{
cannotConvertType("Decimal64");
return false;
}
bool readDecimal128(Decimal128 &, UInt32, UInt32) override
{
cannotConvertType("Decimal128");
return false;
}
bool readAggregateFunction(const AggregateFunctionPtr &, AggregateDataPtr, Arena &) override
{
cannotConvertType("AggregateFunction");
return false;
}
protected:
void cannotConvertType(const String & type_name)
[[noreturn]] void cannotConvertType(const String & type_name)
{
throw Exception(
String("Could not convert type '") + field->type_name() + "' from protobuf field '" + field->name() + "' to data type '"
@ -520,7 +499,7 @@ protected:
ErrorCodes::PROTOBUF_BAD_CAST);
}
void cannotConvertValue(const String & value, const String & type_name)
[[noreturn]] void cannotConvertValue(const String & value, const String & type_name)
{
throw Exception(
"Could not convert value '" + value + "' from protobuf field '" + field->name() + "' to data type '" + type_name + "'",
@ -557,7 +536,6 @@ protected:
catch (...)
{
cannotConvertValue(StringRef(str.data(), str.size()).toString(), TypeName<To>::get());
__builtin_unreachable();
}
}

View File

@ -75,7 +75,7 @@ void registerInputFormatProtobuf(FormatFactory & factory)
const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<ProtobufRowInputStream>(buf, sample, FormatSchemaInfo(context, "proto")),
std::make_shared<ProtobufRowInputStream>(buf, sample, FormatSchemaInfo(context, "Protobuf")),
sample, max_block_size, settings);
});
}

View File

@ -38,7 +38,7 @@ void registerOutputFormatProtobuf(FormatFactory & factory)
"Protobuf", [](WriteBuffer & buf, const Block & header, const Context & context, const FormatSettings &)
{
return std::make_shared<BlockOutputStreamFromRowOutputStream>(
std::make_shared<ProtobufRowOutputStream>(buf, header, FormatSchemaInfo(context, "proto")), header);
std::make_shared<ProtobufRowOutputStream>(buf, header, FormatSchemaInfo(context, "Protobuf")), header);
});
}

View File

@ -334,14 +334,14 @@ public:
virtual void writeAggregateFunction(const AggregateFunctionPtr &, ConstAggregateDataPtr) override { cannotConvertType("AggregateFunction"); }
protected:
void cannotConvertType(const String & type_name)
[[noreturn]] void cannotConvertType(const String & type_name)
{
throw Exception(
"Could not convert data type '" + type_name + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")",
ErrorCodes::PROTOBUF_BAD_CAST);
}
void cannotConvertValue(const String & value)
[[noreturn]] void cannotConvertValue(const String & value)
{
throw Exception(
"Could not convert value '" + value + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")",

View File

@ -423,7 +423,7 @@ inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, Read
/** Throw exception with verbose message when string value is not parsed completely.
*/
void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result);
[[noreturn]] void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result);
enum class ConvertFromStringExceptionMode

View File

@ -119,6 +119,8 @@ struct IntegerRoundingComputation
return x;
}
}
__builtin_unreachable();
}
static ALWAYS_INLINE T compute(T x, T scale)
@ -132,6 +134,8 @@ struct IntegerRoundingComputation
case ScaleMode::Negative:
return computeImpl(x, scale);
}
__builtin_unreachable();
}
static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out)

View File

@ -40,6 +40,7 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
extern const int TOO_MANY_BYTES;
extern const int NOT_IMPLEMENTED;
extern const int HYPERSCAN_CANNOT_SCAN_TEXT;
}
/// Is the LIKE expression reduced to finding a substring in a string?
@ -289,10 +290,10 @@ struct MultiMatchAnyImpl
#if USE_HYPERSCAN
const auto & hyperscan_regex = MultiRegexps::get<FindAnyIndex, MultiSearchDistance>(needles, edit_distance);
hs_scratch_t * scratch = nullptr;
hs_error_t err = hs_alloc_scratch(hyperscan_regex->get(), &scratch);
hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch);
if (err != HS_SUCCESS)
throw Exception("Could not allocate scratch space for hyperscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
throw Exception("Could not clone scratch space for hyperscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
MultiRegexps::ScratchPtr smart_scratch(scratch);
@ -316,14 +317,16 @@ struct MultiMatchAnyImpl
if (length > std::numeric_limits<UInt32>::max())
throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES);
res[i] = 0;
hs_scan(
hyperscan_regex->get(),
err = hs_scan(
hyperscan_regex->getDB(),
reinterpret_cast<const char *>(haystack_data.data()) + offset,
length,
0,
smart_scratch.get(),
on_match,
&res[i]);
if (err != HS_SUCCESS)
throw Exception("Failed to scan with hyperscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT);
offset = haystack_offsets[i];
}
#else

View File

@ -173,10 +173,7 @@ struct PositionImpl
/// We check that the entry does not pass through the boundaries of strings.
if (pos + needle.size() < begin + offsets[i])
{
size_t prev_offset = i != 0 ? offsets[i - 1] : 0;
res[i] = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + prev_offset), reinterpret_cast<const char *>(pos));
}
res[i] = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + offsets[i - 1]), reinterpret_cast<const char *>(pos));
else
res[i] = 0;
@ -306,7 +303,8 @@ struct MultiSearchAllPositionsImpl
const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res)
{
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 {
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
{
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
};
Impl::createMultiSearcherInBigHaystack(needles).searchAllPositions(haystack_data, haystack_offsets, res_callback, res);
@ -341,7 +339,8 @@ struct MultiSearchFirstPositionImpl
const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res)
{
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 {
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
{
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
};
Impl::createMultiSearcherInBigHaystack(needles).searchFirstPosition(haystack_data, haystack_offsets, res_callback, res);

View File

@ -520,7 +520,7 @@ void resizeDynamicSize(ArraySource && array_source, ValueSource && value_source,
while (!sink.isEnd())
{
size_t row_num = array_source.rowNum();
bool has_size = !size_null_map || (size_null_map && (*size_null_map)[row_num]);
bool has_size = !size_null_map || (*size_null_map)[row_num];
if (has_size)
{

View File

@ -1,6 +1,8 @@
#pragma once
#include <map>
#include <memory>
#include <mutex>
#include <optional>
#include <string>
#include <utility>
@ -11,6 +13,7 @@
#include <Common/ProfileEvents.h>
#include <common/StringRef.h>
#include <Common/config.h>
#if USE_HYPERSCAN
# if __has_include(<hs/hs.h>)
@ -84,24 +87,32 @@ namespace MultiRegexps
};
using CompilerError = std::unique_ptr<hs_compile_error_t, HyperscanDeleter<decltype(&hs_free_compile_error), &hs_free_compile_error>>;
using ScratchPtr = std::unique_ptr<hs_scratch_t, DB::MultiRegexps::HyperscanDeleter<decltype(&hs_free_scratch), &hs_free_scratch>>;
using Regexps = std::unique_ptr<hs_database_t, HyperscanDeleter<decltype(&hs_free_database), &hs_free_database>>;
using ScratchPtr = std::unique_ptr<hs_scratch_t, HyperscanDeleter<decltype(&hs_free_scratch), &hs_free_scratch>>;
using DataBasePtr = std::unique_ptr<hs_database_t, HyperscanDeleter<decltype(&hs_free_database), &hs_free_database>>;
using Pool = ObjectPoolMap<Regexps, std::pair<std::vector<String>, std::optional<UInt32>>>;
/// If CompileForEditDistance is False, edit_distance must be nullopt
template <bool FindAnyIndex, bool CompileForEditDistance>
inline Pool::Pointer get(const std::vector<StringRef> & patterns, std::optional<UInt32> edit_distance)
/// Database is thread safe across multiple threads and Scratch is not but we can copy it whenever we use it in the searcher
class Regexps
{
/// C++11 has thread-safe function-local statics on most modern compilers.
static Pool known_regexps; /// Different variables for different pattern parameters.
public:
Regexps(hs_database_t * db_, hs_scratch_t * scratch_) : db{db_}, scratch{scratch_} {}
std::vector<String> str_patterns;
str_patterns.reserve(patterns.size());
for (const StringRef & ref : patterns)
str_patterns.push_back(ref.toString());
hs_database_t * getDB() const { return db.get(); }
hs_scratch_t * getScratch() const { return scratch.get(); }
private:
DataBasePtr db;
ScratchPtr scratch;
};
return known_regexps.get({str_patterns, edit_distance}, [&str_patterns, edit_distance]
struct Pool
{
/// Mutex for finding in map
std::mutex mutex;
/// Patterns + possible edit_distance to database and scratch
std::map<std::pair<std::vector<String>, std::optional<UInt32>>, Regexps> storage;
};
template <bool FindAnyIndex, bool CompileForEditDistance>
inline Regexps constructRegexps(const std::vector<String> & str_patterns, std::optional<UInt32> edit_distance)
{
(void)edit_distance;
/// Common pointers
@ -183,8 +194,39 @@ namespace MultiRegexps
ProfileEvents::increment(ProfileEvents::RegexpCreated);
return new Regexps{db};
});
hs_scratch_t * scratch = nullptr;
err = hs_alloc_scratch(db, &scratch);
if (err != HS_SUCCESS)
throw Exception("Could not allocate scratch space for hyperscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
return Regexps{db, scratch};
}
/// If CompileForEditDistance is False, edit_distance must be nullopt
template <bool FindAnyIndex, bool CompileForEditDistance>
inline Regexps * get(const std::vector<StringRef> & patterns, std::optional<UInt32> edit_distance)
{
/// C++11 has thread-safe function-local statics on most modern compilers.
static Pool known_regexps; /// Different variables for different pattern parameters.
std::vector<String> str_patterns;
str_patterns.reserve(patterns.size());
for (const StringRef & ref : patterns)
str_patterns.push_back(ref.toString());
std::unique_lock lock(known_regexps.mutex);
auto it = known_regexps.storage.find({str_patterns, edit_distance});
if (known_regexps.storage.end() == it)
it = known_regexps.storage.emplace(
std::pair{str_patterns, edit_distance},
constructRegexps<FindAnyIndex, CompileForEditDistance>(str_patterns, edit_distance)).first;
lock.unlock();
return &it->second;
}
}

View File

@ -56,6 +56,7 @@ private:
struct UnpackedArrays
{
size_t base_rows = 0;
std::vector<char> is_const;
std::vector<const NullMap *> null_maps;
std::vector<const ColumnArray::ColumnOffsets::Container *> offsets;
@ -246,6 +247,8 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(con
arrays.offsets.resize(columns_number);
arrays.nested_columns.resize(columns_number);
bool all_const = true;
for (auto i : ext::range(0, columns_number))
{
auto argument_column = columns[i].get();
@ -257,6 +260,9 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(con
if (auto argument_column_array = typeid_cast<const ColumnArray *>(argument_column))
{
if (!arrays.is_const[i])
all_const = false;
arrays.offsets[i] = &argument_column_array->getOffsets();
arrays.nested_columns[i] = &argument_column_array->getData();
if (auto column_nullable = typeid_cast<const ColumnNullable *>(arrays.nested_columns[i]))
@ -269,6 +275,25 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(con
throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR};
}
if (all_const)
{
arrays.base_rows = arrays.offsets.front()->size();
}
else
{
for (auto i : ext::range(0, columns_number))
{
if (arrays.is_const[i])
continue;
size_t rows = arrays.offsets[i]->size();
if (arrays.base_rows == 0 && rows > 0)
arrays.base_rows = rows;
else if (arrays.base_rows != rows)
throw Exception("Non-const array columns in function " + getName() + "should have same rows", ErrorCodes::LOGICAL_ERROR);
}
}
return arrays;
}
@ -277,7 +302,7 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar
const auto & return_type = block.getByPosition(result).type;
auto return_type_array = checkAndGetDataType<DataTypeArray>(return_type.get());
if (!return_type)
if (!return_type_array)
throw Exception{"Return type for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR};
const auto & nested_return_type = return_type_array->getNestedType();
@ -352,7 +377,7 @@ template <typename Map, typename ColumnType, bool is_numeric_column>
ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, MutableColumnPtr result_data_ptr)
{
auto args = arrays.nested_columns.size();
auto rows = arrays.offsets.front()->size();
auto rows = arrays.base_rows;
bool all_nullable = true;
@ -392,26 +417,42 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable
for (auto arg : ext::range(0, args))
{
bool current_has_nullable = false;
size_t off = (*arrays.offsets[arg])[row];
size_t off;
// const array has only one row
bool const_arg = arrays.is_const[arg];
if (const_arg)
off = (*arrays.offsets[arg])[0];
else
off = (*arrays.offsets[arg])[row];
for (auto i : ext::range(prev_off[arg], off))
{
if (arrays.null_maps[arg] && (*arrays.null_maps[arg])[i])
current_has_nullable = true;
else
{
typename Map::mapped_type * value = nullptr;
if constexpr (is_numeric_column)
++map[columns[arg]->getElement(i)];
value = &map[columns[arg]->getElement(i)];
else if constexpr (std::is_same<ColumnType, ColumnString>::value || std::is_same<ColumnType, ColumnFixedString>::value)
++map[columns[arg]->getDataAt(i)];
value = &map[columns[arg]->getDataAt(i)];
else
{
const char * data = nullptr;
++map[columns[arg]->serializeValueIntoArena(i, arena, data)];
value = &map[columns[arg]->serializeValueIntoArena(i, arena, data)];
}
if (*value == arg)
++(*value);
}
}
prev_off[arg] = off;
if (const_arg)
prev_off[arg] = 0;
if (!current_has_nullable)
all_has_nullable = false;
}

View File

@ -61,7 +61,7 @@ public:
{
auto array_size = col_num->getInt(i);
if (unlikely(array_size) < 0)
if (unlikely(array_size < 0))
throw Exception("Array size cannot be negative: while executing function " + getName(), ErrorCodes::TOO_LARGE_ARRAY_SIZE);
offset += array_size;

View File

@ -153,7 +153,7 @@ template <typename A, typename B>
struct NumIfImpl<A, B, NumberTraits::Error>
{
private:
static void throw_error()
[[noreturn]] static void throw_error()
{
throw Exception("Internal logic error: invalid types of arguments 2 and 3 of if", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -0,0 +1,330 @@
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStringOrArrayToT.h>
#include <cstring>
#ifdef __SSE4_1__
# include <emmintrin.h>
# include <smmintrin.h>
# include <tmmintrin.h>
#endif
namespace DB
{
/// inspired by https://github.com/cyb70289/utf8/
struct ValidUTF8Impl
{
/*
MIT License
Copyright (c) 2019 Yibo Cai
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all
copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.
*/
/*
* http://www.unicode.org/versions/Unicode6.0.0/ch03.pdf - page 94
*
* Table 3-7. Well-Formed UTF-8 Byte Sequences
*
* +--------------------+------------+-------------+------------+-------------+
* | Code Points | First Byte | Second Byte | Third Byte | Fourth Byte |
* +--------------------+------------+-------------+------------+-------------+
* | U+0000..U+007F | 00..7F | | | |
* +--------------------+------------+-------------+------------+-------------+
* | U+0080..U+07FF | C2..DF | 80..BF | | |
* +--------------------+------------+-------------+------------+-------------+
* | U+0800..U+0FFF | E0 | A0..BF | 80..BF | |
* +--------------------+------------+-------------+------------+-------------+
* | U+1000..U+CFFF | E1..EC | 80..BF | 80..BF | |
* +--------------------+------------+-------------+------------+-------------+
* | U+D000..U+D7FF | ED | 80..9F | 80..BF | |
* +--------------------+------------+-------------+------------+-------------+
* | U+E000..U+FFFF | EE..EF | 80..BF | 80..BF | |
* +--------------------+------------+-------------+------------+-------------+
* | U+10000..U+3FFFF | F0 | 90..BF | 80..BF | 80..BF |
* +--------------------+------------+-------------+------------+-------------+
* | U+40000..U+FFFFF | F1..F3 | 80..BF | 80..BF | 80..BF |
* +--------------------+------------+-------------+------------+-------------+
* | U+100000..U+10FFFF | F4 | 80..8F | 80..BF | 80..BF |
* +--------------------+------------+-------------+------------+-------------+
*/
static inline UInt8 isValidUTF8Naive(const UInt8 * data, UInt64 len)
{
while (len)
{
int bytes;
const UInt8 byte1 = data[0];
/* 00..7F */
if (byte1 <= 0x7F)
{
bytes = 1;
}
/* C2..DF, 80..BF */
else if (len >= 2 && byte1 >= 0xC2 && byte1 <= 0xDF && static_cast<Int8>(data[1]) <= static_cast<Int8>(0xBF))
{
bytes = 2;
}
else if (len >= 3)
{
const UInt8 byte2 = data[1];
bool byte2_ok = static_cast<Int8>(byte2) <= static_cast<Int8>(0xBF);
bool byte3_ok = static_cast<Int8>(data[2]) <= static_cast<Int8>(0xBF);
if (byte2_ok && byte3_ok &&
/* E0, A0..BF, 80..BF */
((byte1 == 0xE0 && byte2 >= 0xA0) ||
/* E1..EC, 80..BF, 80..BF */
(byte1 >= 0xE1 && byte1 <= 0xEC) ||
/* ED, 80..9F, 80..BF */
(byte1 == 0xED && byte2 <= 0x9F) ||
/* EE..EF, 80..BF, 80..BF */
(byte1 >= 0xEE && byte1 <= 0xEF)))
{
bytes = 3;
}
else if (len >= 4)
{
bool byte4_ok = static_cast<Int8>(data[3]) <= static_cast<Int8>(0xBF);
if (byte2_ok && byte3_ok && byte4_ok &&
/* F0, 90..BF, 80..BF, 80..BF */
((byte1 == 0xF0 && byte2 >= 0x90) ||
/* F1..F3, 80..BF, 80..BF, 80..BF */
(byte1 >= 0xF1 && byte1 <= 0xF3) ||
/* F4, 80..8F, 80..BF, 80..BF */
(byte1 == 0xF4 && byte2 <= 0x8F)))
{
bytes = 4;
}
else
{
return false;
}
}
else
{
return false;
}
}
else
{
return false;
}
len -= bytes;
data += bytes;
}
return true;
}
#ifndef __SSE4_1__
static inline UInt8 isValidUTF8(const UInt8 * data, UInt64 len) { return isValidUTF8Naive(data, len); }
#else
static inline UInt8 isValidUTF8(const UInt8 * data, UInt64 len)
{
/*
* Map high nibble of "First Byte" to legal character length minus 1
* 0x00 ~ 0xBF --> 0
* 0xC0 ~ 0xDF --> 1
* 0xE0 ~ 0xEF --> 2
* 0xF0 ~ 0xFF --> 3
*/
const __m128i first_len_tbl = _mm_setr_epi8(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 2, 3);
/* Map "First Byte" to 8-th item of range table (0xC2 ~ 0xF4) */
const __m128i first_range_tbl = _mm_setr_epi8(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 8, 8, 8, 8);
/*
* Range table, map range index to min and max values
*/
const __m128i range_min_tbl
= _mm_setr_epi8(0x00, 0x80, 0x80, 0x80, 0xA0, 0x80, 0x90, 0x80, 0xC2, 0x7F, 0x7F, 0x7F, 0x7F, 0x7F, 0x7F, 0x7F);
const __m128i range_max_tbl
= _mm_setr_epi8(0x7F, 0xBF, 0xBF, 0xBF, 0xBF, 0x9F, 0xBF, 0x8F, 0xF4, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80);
/*
* Tables for fast handling of four special First Bytes(E0,ED,F0,F4), after
* which the Second Byte are not 80~BF. It contains "range index adjustment".
* +------------+---------------+------------------+----------------+
* | First Byte | original range| range adjustment | adjusted range |
* +------------+---------------+------------------+----------------+
* | E0 | 2 | 2 | 4 |
* +------------+---------------+------------------+----------------+
* | ED | 2 | 3 | 5 |
* +------------+---------------+------------------+----------------+
* | F0 | 3 | 3 | 6 |
* +------------+---------------+------------------+----------------+
* | F4 | 4 | 4 | 8 |
* +------------+---------------+------------------+----------------+
*/
/* index1 -> E0, index14 -> ED */
const __m128i df_ee_tbl = _mm_setr_epi8(0, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3, 0);
/* index1 -> F0, index5 -> F4 */
const __m128i ef_fe_tbl = _mm_setr_epi8(0, 3, 0, 0, 0, 4, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0);
__m128i prev_input = _mm_set1_epi8(0);
__m128i prev_first_len = _mm_set1_epi8(0);
__m128i error = _mm_set1_epi8(0);
auto check_packed = [&](__m128i input) noexcept
{
/* high_nibbles = input >> 4 */
const __m128i high_nibbles = _mm_and_si128(_mm_srli_epi16(input, 4), _mm_set1_epi8(0x0F));
/* first_len = legal character length minus 1 */
/* 0 for 00~7F, 1 for C0~DF, 2 for E0~EF, 3 for F0~FF */
/* first_len = first_len_tbl[high_nibbles] */
__m128i first_len = _mm_shuffle_epi8(first_len_tbl, high_nibbles);
/* First Byte: set range index to 8 for bytes within 0xC0 ~ 0xFF */
/* range = first_range_tbl[high_nibbles] */
__m128i range = _mm_shuffle_epi8(first_range_tbl, high_nibbles);
/* Second Byte: set range index to first_len */
/* 0 for 00~7F, 1 for C0~DF, 2 for E0~EF, 3 for F0~FF */
/* range |= (first_len, prev_first_len) << 1 byte */
range = _mm_or_si128(range, _mm_alignr_epi8(first_len, prev_first_len, 15));
/* Third Byte: set range index to saturate_sub(first_len, 1) */
/* 0 for 00~7F, 0 for C0~DF, 1 for E0~EF, 2 for F0~FF */
__m128i tmp1;
__m128i tmp2;
/* tmp1 = saturate_sub(first_len, 1) */
tmp1 = _mm_subs_epu8(first_len, _mm_set1_epi8(1));
/* tmp2 = saturate_sub(prev_first_len, 1) */
tmp2 = _mm_subs_epu8(prev_first_len, _mm_set1_epi8(1));
/* range |= (tmp1, tmp2) << 2 bytes */
range = _mm_or_si128(range, _mm_alignr_epi8(tmp1, tmp2, 14));
/* Fourth Byte: set range index to saturate_sub(first_len, 2) */
/* 0 for 00~7F, 0 for C0~DF, 0 for E0~EF, 1 for F0~FF */
/* tmp1 = saturate_sub(first_len, 2) */
tmp1 = _mm_subs_epu8(first_len, _mm_set1_epi8(2));
/* tmp2 = saturate_sub(prev_first_len, 2) */
tmp2 = _mm_subs_epu8(prev_first_len, _mm_set1_epi8(2));
/* range |= (tmp1, tmp2) << 3 bytes */
range = _mm_or_si128(range, _mm_alignr_epi8(tmp1, tmp2, 13));
/*
* Now we have below range indices caluclated
* Correct cases:
* - 8 for C0~FF
* - 3 for 1st byte after F0~FF
* - 2 for 1st byte after E0~EF or 2nd byte after F0~FF
* - 1 for 1st byte after C0~DF or 2nd byte after E0~EF or
* 3rd byte after F0~FF
* - 0 for others
* Error cases:
* 9,10,11 if non ascii First Byte overlaps
* E.g., F1 80 C2 90 --> 8 3 10 2, where 10 indicates error
*/
/* Adjust Second Byte range for special First Bytes(E0,ED,F0,F4) */
/* Overlaps lead to index 9~15, which are illegal in range table */
__m128i shift1, pos, range2;
/* shift1 = (input, prev_input) << 1 byte */
shift1 = _mm_alignr_epi8(input, prev_input, 15);
pos = _mm_sub_epi8(shift1, _mm_set1_epi8(0xEF));
/*
* shift1: | EF F0 ... FE | FF 00 ... ... DE | DF E0 ... EE |
* pos: | 0 1 15 | 16 17 239| 240 241 255|
* pos-240: | 0 0 0 | 0 0 0 | 0 1 15 |
* pos+112: | 112 113 127| >= 128 | >= 128 |
*/
tmp1 = _mm_subs_epu8(pos, _mm_set1_epi8(240));
range2 = _mm_shuffle_epi8(df_ee_tbl, tmp1);
tmp2 = _mm_adds_epu8(pos, _mm_set1_epi8(112));
range2 = _mm_add_epi8(range2, _mm_shuffle_epi8(ef_fe_tbl, tmp2));
range = _mm_add_epi8(range, range2);
/* Load min and max values per calculated range index */
__m128i minv = _mm_shuffle_epi8(range_min_tbl, range);
__m128i maxv = _mm_shuffle_epi8(range_max_tbl, range);
/* Check value range */
error = _mm_or_si128(error, _mm_cmplt_epi8(input, minv));
error = _mm_or_si128(error, _mm_cmpgt_epi8(input, maxv));
prev_input = input;
prev_first_len = first_len;
data += 16;
len -= 16;
};
while (len >= 16)
check_packed(_mm_loadu_si128(reinterpret_cast<const __m128i *>(data)));
/// 0 <= len <= 15 for now. Reading data from data - 1 because of right padding of 15 and left padding
/// Then zero some bytes from the unknown memory and check again.
alignas(16) char buf[32];
_mm_store_si128(reinterpret_cast<__m128i *>(buf), _mm_loadu_si128(reinterpret_cast<const __m128i *>(data - 1)));
memset(buf + len + 1, 0, 16);
check_packed(_mm_loadu_si128(reinterpret_cast<__m128i *>(buf + 1)));
/* Reduce error vector, error_reduced = 0xFFFF if error == 0 */
return _mm_movemask_epi8(_mm_cmpeq_epi8(error, _mm_set1_epi8(0))) == 0xFFFF;
}
#endif
static constexpr bool is_fixed_to_constant = false;
static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray<UInt8> & res)
{
size_t size = offsets.size();
size_t prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
res[i] = isValidUTF8(data.data() + prev_offset, offsets[i] - 1 - prev_offset);
prev_offset = offsets[i];
}
}
static void vector_fixed_to_constant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt8 & /*res*/) {}
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt8> & res)
{
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
res[i] = isValidUTF8(data.data() + i * n, n);
}
static void array(const ColumnString::Offsets &, PaddedPODArray<UInt8> &)
{
throw Exception("Cannot apply function isValidUTF8 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
struct NameValidUTF8
{
static constexpr auto name = "isValidUTF8";
};
using FunctionValidUTF8 = FunctionStringOrArrayToT<ValidUTF8Impl, NameValidUTF8, UInt8>;
void registerFunctionValidUTF8(FunctionFactory & factory)
{
factory.registerFunction<FunctionValidUTF8>();
}
}

View File

@ -17,15 +17,15 @@ struct ExtractQueryStringAndFragment
res_data = data;
res_size = 0;
Pos pos = data;
Pos end = pos + size;
Pos end = data + size;
Pos pos;
if (end != (pos = find_first_symbols<'?'>(pos, end)))
if (end != (pos = find_first_symbols<'?'>(data, end)))
{
res_data = pos + (without_leading_char ? 1 : 0);
res_size = end - res_data;
}
else if (end != (pos = find_first_symbols<'#'>(pos, end)))
else if (end != (pos = find_first_symbols<'#'>(data, end)))
{
res_data = pos;
res_size = end - res_data;

View File

@ -9,6 +9,7 @@ void registerFunctionEmpty(FunctionFactory &);
void registerFunctionNotEmpty(FunctionFactory &);
void registerFunctionLength(FunctionFactory &);
void registerFunctionLengthUTF8(FunctionFactory &);
void registerFunctionValidUTF8(FunctionFactory &);
void registerFunctionLower(FunctionFactory &);
void registerFunctionUpper(FunctionFactory &);
void registerFunctionLowerUTF8(FunctionFactory &);
@ -36,6 +37,7 @@ void registerFunctionsString(FunctionFactory & factory)
registerFunctionNotEmpty(factory);
registerFunctionLength(factory);
registerFunctionLengthUTF8(factory);
registerFunctionValidUTF8(factory);
registerFunctionLower(factory);
registerFunctionUpper(factory);
registerFunctionLowerUTF8(factory);

View File

@ -7,6 +7,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BROTLI_READ_FAILED;
}
class BrotliReadBuffer::BrotliStateWrapper
{
public:
@ -29,7 +35,7 @@ public:
BrotliReadBuffer::BrotliReadBuffer(ReadBuffer &in_, size_t buf_size, char *existing_memory, size_t alignment)
: BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment)
, in(in_)
, brotli(new BrotliStateWrapper())
, brotli(std::make_unique<BrotliStateWrapper>())
, in_available(0)
, in_data(nullptr)
, out_capacity(0)
@ -56,7 +62,7 @@ bool BrotliReadBuffer::nextImpl()
if (brotli->result == BROTLI_DECODER_RESULT_NEEDS_MORE_INPUT && (!in_available || in.eof()))
{
throw Exception(std::string("brotli decode error"), ErrorCodes::CANNOT_READ_ALL_DATA);
throw Exception("brotli decode error", ErrorCodes::BROTLI_READ_FAILED);
}
out_capacity = internal_buffer.size();
@ -76,13 +82,13 @@ bool BrotliReadBuffer::nextImpl()
}
else
{
throw Exception(std::string("brotli decode error"), ErrorCodes::CANNOT_READ_ALL_DATA);
throw Exception("brotli decode error", ErrorCodes::BROTLI_READ_FAILED);
}
}
if (brotli->result == BROTLI_DECODER_RESULT_ERROR)
{
throw Exception(std::string("brotli decode error"), ErrorCodes::CANNOT_READ_ALL_DATA);
throw Exception("brotli decode error", ErrorCodes::BROTLI_READ_FAILED);
}
return true;

View File

@ -34,5 +34,6 @@ private:
bool eof;
};
}

View File

@ -0,0 +1,126 @@
#include <Common/config.h>
#if USE_BROTLI
#include <IO/BrotliWriteBuffer.h>
#include <brotli/encode.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BROTLI_WRITE_FAILED;
}
class BrotliWriteBuffer::BrotliStateWrapper
{
public:
BrotliStateWrapper()
: state(BrotliEncoderCreateInstance(nullptr, nullptr, nullptr))
{
}
~BrotliStateWrapper()
{
BrotliEncoderDestroyInstance(state);
}
public:
BrotliEncoderState * state;
};
BrotliWriteBuffer::BrotliWriteBuffer(WriteBuffer & out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment)
: BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment)
, brotli(std::make_unique<BrotliStateWrapper>())
, in_available(0)
, in_data(nullptr)
, out_capacity(0)
, out_data(nullptr)
, out(out_)
{
BrotliEncoderSetParameter(brotli->state, BROTLI_PARAM_QUALITY, static_cast<uint32_t>(compression_level));
// Set LZ77 window size. According to brotli sources default value is 24 (c/tools/brotli.c:81)
BrotliEncoderSetParameter(brotli->state, BROTLI_PARAM_LGWIN, 24);
}
BrotliWriteBuffer::~BrotliWriteBuffer()
{
finish();
}
void BrotliWriteBuffer::nextImpl()
{
if (!offset())
{
return;
}
in_data = reinterpret_cast<unsigned char *>(working_buffer.begin());
in_available = offset();
do
{
out.nextIfAtEnd();
out_data = reinterpret_cast<unsigned char *>(out.position());
out_capacity = out.buffer().end() - out.position();
int result = BrotliEncoderCompressStream(
brotli->state,
in_available ? BROTLI_OPERATION_PROCESS : BROTLI_OPERATION_FINISH,
&in_available,
&in_data,
&out_capacity,
&out_data,
nullptr);
out.position() = out.buffer().end() - out_capacity;
if (result == 0)
{
throw Exception("brotli compress failed", ErrorCodes::BROTLI_WRITE_FAILED);
}
}
while (in_available > 0 || out_capacity == 0);
}
void BrotliWriteBuffer::finish()
{
if (finished)
return;
next();
while (true)
{
out.nextIfAtEnd();
out_data = reinterpret_cast<unsigned char *>(out.position());
out_capacity = out.buffer().end() - out.position();
int result = BrotliEncoderCompressStream(
brotli->state,
BROTLI_OPERATION_FINISH,
&in_available,
&in_data,
&out_capacity,
&out_data,
nullptr);
out.position() = out.buffer().end() - out_capacity;
if (BrotliEncoderIsFinished(brotli->state))
{
finished = true;
return;
}
if (result == 0)
{
throw Exception("brotli compress failed", ErrorCodes::BROTLI_WRITE_FAILED);
}
}
}
}
#endif

View File

@ -0,0 +1,40 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <IO/BufferWithOwnMemory.h>
namespace DB
{
class BrotliWriteBuffer : public BufferWithOwnMemory<WriteBuffer>
{
public:
BrotliWriteBuffer(
WriteBuffer & out_,
int compression_level,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);
~BrotliWriteBuffer() override;
void finish();
private:
void nextImpl() override;
class BrotliStateWrapper;
std::unique_ptr<BrotliStateWrapper> brotli;
size_t in_available;
const uint8_t * in_data;
size_t out_capacity;
uint8_t * out_data;
WriteBuffer & out;
bool finished = false;
};
}

View File

@ -24,7 +24,8 @@ namespace DB
* Differs in that is doesn't do unneeded memset. (And also tries to do as little as possible.)
* Also allows to allocate aligned piece of memory (to use with O_DIRECT, for example).
*/
struct Memory : boost::noncopyable, Allocator<false>
template <typename Allocator = Allocator<false>>
struct Memory : boost::noncopyable, Allocator
{
/// Padding is needed to allow usage of 'memcpySmallAllowReadWriteOverflow15' function with this buffer.
static constexpr size_t pad_right = 15;
@ -136,7 +137,7 @@ template <typename Base>
class BufferWithOwnMemory : public Base
{
protected:
Memory memory;
Memory<> memory;
public:
/// If non-nullptr 'existing_memory' is passed, then buffer will not create its own memory and will use existing_memory without ownership.
BufferWithOwnMemory(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0)

View File

@ -3,7 +3,7 @@
namespace DB
{
enum class ZlibCompressionMethod
enum class CompressionMethod
{
/// DEFLATE compression with gzip header and CRC32 checksum.
/// This option corresponds to files produced by gzip(1) or HTTP Content-Encoding: gzip.
@ -11,6 +11,7 @@ enum class ZlibCompressionMethod
/// DEFLATE compression with zlib header and Adler32 checksum.
/// This option corresponds to HTTP Content-Encoding: deflate.
Zlib,
Brotli,
};
}

View File

@ -36,7 +36,7 @@ protected:
return false;
BufferBase::set(buffer->position(), buffer->available(), 0);
put_delimiter = true;
put_delimiter = (delimiter != 0);
}
return true;

View File

@ -179,7 +179,7 @@ private:
*/
virtual bool nextImpl() { return false; }
void throwReadAfterEOF()
[[noreturn]] void throwReadAfterEOF()
{
throw Exception("Attempt to read after eof", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF);
}

View File

@ -187,6 +187,9 @@ off_t ReadBufferAIO::doSeek(off_t off, int whence)
pos = working_buffer.end();
first_unread_pos_in_file = new_pos_in_file;
/// If we go back, than it's not eof
is_eof = false;
/// We can not use the result of the current asynchronous request.
skip();
}

View File

@ -43,6 +43,7 @@ protected:
ProfileCallback profile_callback;
clockid_t clock_type;
/// Children implementation should be able to seek backwards
virtual off_t doSeek(off_t off, int whence) = 0;
};

View File

@ -164,7 +164,7 @@ void readVectorBinary(std::vector<T> & v, ReadBuffer & buf, size_t MAX_VECTOR_SI
void assertString(const char * s, ReadBuffer & buf);
void assertEOF(ReadBuffer & buf);
void throwAtAssertionFailed(const char * s, ReadBuffer & buf);
[[noreturn]] void throwAtAssertionFailed(const char * s, ReadBuffer & buf);
inline void assertChar(char symbol, ReadBuffer & buf)
{

View File

@ -20,8 +20,9 @@ namespace DB
struct UncompressedCacheCell
{
Memory data;
Memory<> data;
size_t compressed_size;
UInt32 additional_bytes;
};
struct UncompressedSizeWeightFunction

View File

@ -113,7 +113,7 @@ readVarUInt(T & x, ReadBuffer & istr)
}
inline void throwReadAfterEOF()
[[noreturn]] inline void throwReadAfterEOF()
{
throw Exception("Attempt to read after eof", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF);
}

View File

@ -76,34 +76,47 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
{
if (compress)
{
if (compression_method == ZlibCompressionMethod::Gzip)
if (compression_method == CompressionMethod::Gzip)
{
#if defined(POCO_CLICKHOUSE_PATCH)
*response_header_ostr << "Content-Encoding: gzip\r\n";
#else
response.set("Content-Encoding", "gzip");
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf;
}
else if (compression_method == ZlibCompressionMethod::Zlib)
else if (compression_method == CompressionMethod::Zlib)
{
#if defined(POCO_CLICKHOUSE_PATCH)
*response_header_ostr << "Content-Encoding: deflate\r\n";
#else
response.set("Content-Encoding", "deflate");
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf;
}
else if (compression_method == CompressionMethod::Brotli)
{
#if defined(POCO_CLICKHOUSE_PATCH)
*response_header_ostr << "Content-Encoding: br\r\n";
#else
response.set("Content-Encoding", "br");
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
brotli_buf.emplace(*out_raw, compression_level, working_buffer.size(), working_buffer.begin());
out = &*brotli_buf;
}
else
throw Exception("Logical error: unknown compression method passed to WriteBufferFromHTTPServerResponse",
ErrorCodes::LOGICAL_ERROR);
/// Use memory allocated for the outer buffer in the buffer pointed to by out. This avoids extra allocation and copy.
#if !defined(POCO_CLICKHOUSE_PATCH)
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf;
}
else
{
@ -133,7 +146,7 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse(
Poco::Net::HTTPServerResponse & response_,
unsigned keep_alive_timeout_,
bool compress_,
ZlibCompressionMethod compression_method_,
CompressionMethod compression_method_,
size_t size)
: BufferWithOwnMemory<WriteBuffer>(size)
, request(request_)

View File

@ -9,6 +9,7 @@
#include <IO/BufferWithOwnMemory.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/ZlibDeflatingWriteBuffer.h>
#include <IO/BrotliWriteBuffer.h>
#include <IO/HTTPCommon.h>
#include <IO/Progress.h>
#include <Common/NetException.h>
@ -49,7 +50,7 @@ private:
bool add_cors_header = false;
unsigned keep_alive_timeout = 0;
bool compress = false;
ZlibCompressionMethod compression_method;
CompressionMethod compression_method;
int compression_level = Z_DEFAULT_COMPRESSION;
std::ostream * response_body_ostr = nullptr;
@ -60,6 +61,7 @@ private:
std::optional<WriteBufferFromOStream> out_raw;
std::optional<ZlibDeflatingWriteBuffer> deflating_buf;
std::optional<BrotliWriteBuffer> brotli_buf;
WriteBuffer * out = nullptr; /// Uncompressed HTTP body is written to this buffer. Points to out_raw or possibly to deflating_buf.
@ -89,7 +91,7 @@ public:
Poco::Net::HTTPServerResponse & response_,
unsigned keep_alive_timeout_,
bool compress_ = false, /// If true - set Content-Encoding header and compress the result.
ZlibCompressionMethod compression_method_ = ZlibCompressionMethod::Gzip,
CompressionMethod compression_method_ = CompressionMethod::Gzip,
size_t size = DBMS_DEFAULT_BUFFER_SIZE);
/// Writes progess in repeating HTTP headers.

View File

@ -6,7 +6,7 @@ namespace DB
ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer(
WriteBuffer & out_,
ZlibCompressionMethod compression_method,
CompressionMethod compression_method,
int compression_level,
size_t buf_size,
char * existing_memory,
@ -23,7 +23,7 @@ ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer(
zstr.avail_out = 0;
int window_bits = 15;
if (compression_method == ZlibCompressionMethod::Gzip)
if (compression_method == CompressionMethod::Gzip)
{
window_bits += 16;
}

View File

@ -2,7 +2,7 @@
#include <IO/WriteBuffer.h>
#include <IO/BufferWithOwnMemory.h>
#include <IO/ZlibCompressionMethod.h>
#include <IO/CompressionMethod.h>
#include <zlib.h>
@ -21,7 +21,7 @@ class ZlibDeflatingWriteBuffer : public BufferWithOwnMemory<WriteBuffer>
public:
ZlibDeflatingWriteBuffer(
WriteBuffer & out_,
ZlibCompressionMethod compression_method,
CompressionMethod compression_method,
int compression_level,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,

View File

@ -6,7 +6,7 @@ namespace DB
ZlibInflatingReadBuffer::ZlibInflatingReadBuffer(
ReadBuffer & in_,
ZlibCompressionMethod compression_method,
CompressionMethod compression_method,
size_t buf_size,
char * existing_memory,
size_t alignment)
@ -23,7 +23,7 @@ ZlibInflatingReadBuffer::ZlibInflatingReadBuffer(
zstr.avail_out = 0;
int window_bits = 15;
if (compression_method == ZlibCompressionMethod::Gzip)
if (compression_method == CompressionMethod::Gzip)
{
window_bits += 16;
}

View File

@ -2,7 +2,7 @@
#include <IO/ReadBuffer.h>
#include <IO/BufferWithOwnMemory.h>
#include <IO/ZlibCompressionMethod.h>
#include <IO/CompressionMethod.h>
#include <zlib.h>
@ -22,7 +22,7 @@ class ZlibInflatingReadBuffer : public BufferWithOwnMemory<ReadBuffer>
public:
ZlibInflatingReadBuffer(
ReadBuffer & in_,
ZlibCompressionMethod compression_method,
CompressionMethod compression_method,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);

View File

@ -0,0 +1,71 @@
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wundef"
#endif
#include <gtest/gtest.h>
#include <Core/Defines.h>
#include <port/unistd.h>
#include <IO/ReadBufferAIO.h>
#include <fstream>
namespace
{
std::string createTmpFileForEOFtest()
{
char pattern[] = "/tmp/fileXXXXXX";
char * dir = ::mkdtemp(pattern);
return std::string(dir) + "/foo";
}
void prepare_for_eof(std::string & filename, std::string & buf)
{
static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789";
filename = createTmpFileForEOFtest();
size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE;
buf.reserve(n);
for (size_t i = 0; i < n; ++i)
buf += symbols[i % symbols.length()];
std::ofstream out(filename.c_str());
out << buf;
}
}
TEST(ReadBufferAIOTest, TestReadAfterAIO)
{
using namespace DB;
std::string data;
std::string file_path;
prepare_for_eof(file_path, data);
ReadBufferAIO testbuf(file_path);
std::string newdata;
newdata.resize(data.length());
size_t total_read = testbuf.read(newdata.data(), newdata.length());
EXPECT_EQ(total_read, data.length());
EXPECT_TRUE(testbuf.eof());
testbuf.seek(data.length() - 100);
std::string smalldata;
smalldata.resize(100);
size_t read_after_eof = testbuf.read(smalldata.data(), smalldata.size());
EXPECT_EQ(read_after_eof, 100);
EXPECT_TRUE(testbuf.eof());
testbuf.seek(0);
std::string repeatdata;
repeatdata.resize(data.length());
size_t read_after_eof_big = testbuf.read(repeatdata.data(), repeatdata.size());
EXPECT_EQ(read_after_eof_big, data.length());
EXPECT_TRUE(testbuf.eof());
}

View File

@ -23,7 +23,7 @@ try
{
DB::WriteBufferFromFile buf("test_zlib_buffers.gz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC);
DB::ZlibDeflatingWriteBuffer deflating_buf(buf, DB::ZlibCompressionMethod::Gzip, /* compression_level = */ 3);
DB::ZlibDeflatingWriteBuffer deflating_buf(buf, DB::CompressionMethod::Gzip, /* compression_level = */ 3);
stopwatch.restart();
for (size_t i = 0; i < n; ++i)
@ -41,7 +41,7 @@ try
{
DB::ReadBufferFromFile buf("test_zlib_buffers.gz");
DB::ZlibInflatingReadBuffer inflating_buf(buf, DB::ZlibCompressionMethod::Gzip);
DB::ZlibInflatingReadBuffer inflating_buf(buf, DB::CompressionMethod::Gzip);
stopwatch.restart();
for (size_t i = 0; i < n; ++i)

View File

@ -1,7 +1,7 @@
#include "DNSCacheUpdater.h"
#include <Common/DNSResolver.h>
#include <Interpreters/Context.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <Core/BackgroundSchedulePool.h>
#include <Common/ProfileEvents.h>
#include <Poco/Net/NetException.h>
#include <common/logger_useful.h>
@ -16,8 +16,6 @@ namespace ProfileEvents
namespace DB
{
using BackgroundProcessingPoolTaskInfo = BackgroundProcessingPool::TaskInfo;
namespace ErrorCodes
{
extern const int TIMEOUT_EXCEEDED;
@ -56,18 +54,15 @@ static bool isNetworkError()
DNSCacheUpdater::DNSCacheUpdater(Context & context_)
: context(context_), pool(context_.getBackgroundPool())
: context(context_), pool(context_.getSchedulePool())
{
task_handle = pool.addTask([this] () { return run(); });
task_handle = pool.createTask("DNSCacheUpdater", [this]{ run(); });
}
BackgroundProcessingPoolTaskResult DNSCacheUpdater::run()
void DNSCacheUpdater::run()
{
/// TODO: Ensusre that we get global counter (not thread local)
auto num_current_network_exceptions = ProfileEvents::global_counters[ProfileEvents::NetworkErrors].load(std::memory_order_relaxed);
if (num_current_network_exceptions >= last_num_network_erros + min_errors_to_update_cache
&& time(nullptr) > last_update_time + min_update_period_seconds)
if (num_current_network_exceptions >= last_num_network_erros + min_errors_to_update_cache)
{
try
{
@ -77,32 +72,18 @@ BackgroundProcessingPoolTaskResult DNSCacheUpdater::run()
context.reloadClusterConfig();
last_num_network_erros = num_current_network_exceptions;
last_update_time = time(nullptr);
return BackgroundProcessingPoolTaskResult::SUCCESS;
task_handle->scheduleAfter(min_update_period_seconds * 1000);
return;
}
catch (...)
{
/// Do not increment ProfileEvents::NetworkErrors twice
if (isNetworkError())
return BackgroundProcessingPoolTaskResult::ERROR;
throw;
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
/// According to BackgroundProcessingPool logic, if task has done work, it could be executed again immediately.
return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO;
task_handle->scheduleAfter(10 * 1000);
}
DNSCacheUpdater::~DNSCacheUpdater()
{
if (task_handle)
pool.removeTask(task_handle);
task_handle.reset();
}
bool DNSCacheUpdater::incrementNetworkErrorEventsIfNeeded()
{
if (isNetworkError())

View File

@ -4,35 +4,31 @@
#include <ctime>
#include <cstddef>
#include <Core/BackgroundSchedulePool.h>
namespace DB
{
class Context;
class BackgroundProcessingPool;
class BackgroundProcessingPoolTaskInfo;
enum class BackgroundProcessingPoolTaskResult;
/// Add a task to BackgroundProcessingPool that watch for ProfileEvents::NetworkErrors and updates DNS cache if it has increased
class DNSCacheUpdater
{
public:
explicit DNSCacheUpdater(Context & context);
~DNSCacheUpdater();
/// Checks if it is a network error and increments ProfileEvents::NetworkErrors
static bool incrementNetworkErrorEventsIfNeeded();
private:
BackgroundProcessingPoolTaskResult run();
void run();
Context & context;
BackgroundProcessingPool & pool;
std::shared_ptr<BackgroundProcessingPoolTaskInfo> task_handle;
BackgroundSchedulePool & pool;
BackgroundSchedulePoolTaskHolder task_handle;
size_t last_num_network_erros = 0;
time_t last_update_time = 0;
static constexpr size_t min_errors_to_update_cache = 3;
static constexpr time_t min_update_period_seconds = 45;

View File

@ -19,6 +19,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/IColumn.h>
@ -406,7 +407,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr
getRootActions(arguments[i], true, actions);
const std::string & name = arguments[i]->getColumnName();
types[i] = actions->getSampleBlock().getByName(name).type;
types[i] = recursiveRemoveLowCardinality(actions->getSampleBlock().getByName(name).type);
aggregate.argument_names[i] = name;
}
@ -974,19 +975,11 @@ void ExpressionAnalyzer::collectUsedColumns()
RequiredSourceColumnsVisitor::Data columns_context;
RequiredSourceColumnsVisitor(columns_context).visit(query);
NameSet required = columns_context.requiredColumns();
NameSet source_column_names;
for (const auto & column : source_columns)
source_column_names.insert(column.name);
#if 0
std::cerr << "Query: " << query << std::endl;
std::cerr << "CTX: " << columns_context << std::endl;
std::cerr << "source_columns: ";
for (const auto & name : source_columns)
std::cerr << "'" << name.name << "' ";
std::cerr << "required: ";
for (const auto & pr : required)
std::cerr << "'" << pr.first << "' ";
std::cerr << std::endl;
#endif
NameSet required = columns_context.requiredColumns();
if (columns_context.has_table_join)
{
@ -1013,10 +1006,10 @@ void ExpressionAnalyzer::collectUsedColumns()
}
}
NameSet array_join_sources;
if (columns_context.has_array_join)
{
/// Insert the columns required for the ARRAY JOIN calculation into the required columns list.
NameSet array_join_sources;
for (const auto & result_source : syntax->array_join_result_to_source)
array_join_sources.insert(result_source.second);
@ -1063,15 +1056,39 @@ void ExpressionAnalyzer::collectUsedColumns()
if (!unknown_required_source_columns.empty())
{
std::stringstream ss;
ss << "query: '" << query << "' ";
ss << columns_context;
ss << "source_columns: ";
for (const auto & name : source_columns)
ss << "'" << name.name << "' ";
ss << "Missing columns:";
for (const auto & name : unknown_required_source_columns)
ss << " '" << name << "'";
ss << " while processing query: '" << query << "'";
throw Exception("Unknown identifier: " + *unknown_required_source_columns.begin()
+ (select_query && !select_query->tables ? ". Note that there are no tables (FROM clause) in your query" : "")
+ ", context: " + ss.str(), ErrorCodes::UNKNOWN_IDENTIFIER);
ss << ", required columns:";
for (const auto & name : columns_context.requiredColumns())
ss << " '" << name << "'";
if (!source_column_names.empty())
{
ss << ", source columns:";
for (const auto & name : source_column_names)
ss << " '" << name << "'";
}
else
ss << ", no source columns";
if (columns_context.has_table_join)
{
ss << ", joined columns:";
for (const auto & column : analyzedJoin().available_joined_columns)
ss << " '" << column.name_and_type.name << "'";
}
if (!array_join_sources.empty())
{
ss << ", arrayJoin columns:";
for (const auto & name : array_join_sources)
ss << " '" << name << "'";
}
throw Exception(ss.str(), ErrorCodes::UNKNOWN_IDENTIFIER);
}
}

View File

@ -8,7 +8,6 @@
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnVector.h>
#include <Common/LRUCache.h>
#include <Common/MemoryTracker.h>
#include <Common/typeid_cast.h>
#include <Common/ProfileEvents.h>
#include <Common/Stopwatch.h>

View File

@ -46,48 +46,36 @@ public:
has_global_subqueries(has_global_subqueries_)
{}
void addExternalStorage(ASTPtr & subquery_or_table_name_or_table_expression)
void addExternalStorage(ASTPtr & ast, bool set_alias = false)
{
/// With nondistributed queries, creating temporary tables does not make sense.
if (!is_remote)
return;
ASTPtr subquery;
ASTPtr table_name;
ASTPtr subquery_or_table_name;
bool is_table = false;
ASTPtr subquery_or_table_name = ast; /// ASTIdentifier | ASTSubquery | ASTTableExpression
if (subquery_or_table_name_or_table_expression->as<ASTIdentifier>())
{
table_name = subquery_or_table_name_or_table_expression;
subquery_or_table_name = table_name;
}
else if (const auto * ast_table_expr = subquery_or_table_name_or_table_expression->as<ASTTableExpression>())
if (const auto * ast_table_expr = ast->as<ASTTableExpression>())
{
subquery_or_table_name = ast_table_expr->subquery;
if (ast_table_expr->database_and_table_name)
{
table_name = ast_table_expr->database_and_table_name;
subquery_or_table_name = table_name;
}
else if (ast_table_expr->subquery)
{
subquery = ast_table_expr->subquery;
subquery_or_table_name = subquery;
subquery_or_table_name = ast_table_expr->database_and_table_name;
is_table = true;
}
}
else if (subquery_or_table_name_or_table_expression->as<ASTSubquery>())
{
subquery = subquery_or_table_name_or_table_expression;
subquery_or_table_name = subquery;
}
else if (ast->as<ASTIdentifier>())
is_table = true;
if (!subquery_or_table_name)
throw Exception("Logical error: unknown AST element passed to ExpressionAnalyzer::addExternalStorage method",
ErrorCodes::LOGICAL_ERROR);
if (table_name)
if (is_table)
{
/// If this is already an external table, you do not need to add anything. Just remember its presence.
if (external_tables.end() != external_tables.find(*getIdentifierName(table_name)))
if (external_tables.end() != external_tables.find(*getIdentifierName(subquery_or_table_name)))
return;
}
@ -114,8 +102,16 @@ public:
*/
auto database_and_table_name = createTableIdentifier("", external_table_name);
if (set_alias)
{
String alias = subquery_or_table_name->tryGetAlias();
if (auto * table_name = subquery_or_table_name->as<ASTIdentifier>())
if (alias.empty())
alias = table_name->shortName();
database_and_table_name->setAlias(alias);
}
if (auto * ast_table_expr = subquery_or_table_name_or_table_expression->as<ASTTableExpression>())
if (auto * ast_table_expr = ast->as<ASTTableExpression>())
{
ast_table_expr->subquery.reset();
ast_table_expr->database_and_table_name = database_and_table_name;
@ -124,7 +120,7 @@ public:
ast_table_expr->children.emplace_back(database_and_table_name);
}
else
subquery_or_table_name_or_table_expression = database_and_table_name;
ast = database_and_table_name;
external_tables[external_table_name] = external_storage;
subqueries_for_sets[external_table_name].source = interpreter->execute().in;
@ -170,7 +166,7 @@ private:
{
if (table_elem.table_join && table_elem.table_join->as<ASTTableJoin &>().locality == ASTTableJoin::Locality::Global)
{
data.addExternalStorage(table_elem.table_expression);
data.addExternalStorage(table_elem.table_expression, true);
data.has_global_subqueries = true;
}
}

View File

@ -203,7 +203,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (settings.allow_experimental_multiple_joins_emulation)
{
JoinToSubqueryTransformVisitor::Data join_to_subs_data;
JoinToSubqueryTransformVisitor::Data join_to_subs_data{context};
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr);
}

View File

@ -298,12 +298,17 @@ void Join::setSampleBlock(const Block & block)
if (kind != ASTTableJoin::Kind::Left and kind != ASTTableJoin::Kind::Inner)
throw Exception("ASOF only supports LEFT and INNER as base joins", ErrorCodes::NOT_IMPLEMENTED);
if (key_columns.back()->sizeOfValueIfFixed() != sizeof(ASOFTimeType))
const IColumn * asof_column = key_columns.back();
size_t asof_size;
asof_type = AsofRowRefs::getTypeSize(asof_column, asof_size);
if (!asof_type)
{
std::string msg = "ASOF join column needs to have size ";
msg += std::to_string(sizeof(ASOFTimeType));
std::string msg = "ASOF join not supported for type";
msg += asof_column->getFamilyName();
throw Exception(msg, ErrorCodes::BAD_TYPE_OF_FIELD);
}
key_columns.pop_back();
if (key_columns.empty())
@ -314,7 +319,7 @@ void Join::setSampleBlock(const Block & block)
/// Therefore, add it back in such that it can be extracted appropriately from the full stored
/// key_columns and key_sizes
init(chooseMethod(key_columns, key_sizes));
key_sizes.push_back(sizeof(ASOFTimeType));
key_sizes.push_back(asof_size);
}
else
{
@ -325,6 +330,9 @@ void Join::setSampleBlock(const Block & block)
sample_block_with_columns_to_add = materializeBlock(block);
blocklist_sample = Block(block.getColumnsWithTypeAndName());
prepareBlockListStructure(blocklist_sample);
/// Move from `sample_block_with_columns_to_add` key columns to `sample_block_with_keys`, keeping the order.
size_t pos = 0;
while (pos < sample_block_with_columns_to_add.columns())
@ -357,47 +365,19 @@ void Join::setSampleBlock(const Block & block)
convertColumnToNullable(sample_block_with_columns_to_add.getByPosition(i));
}
void Join::TSRowRef::insert(Join::ASOFTimeType t, const Block * block, size_t row_num)
{
ts.insert(std::pair(t, RowRef(block, row_num)));
}
std::string Join::TSRowRef::dumpStructure() const
{
std::stringstream ss;
for (auto const& x : ts)
{
ss << "(t=" << x.first << " row_num=" << x.second.row_num << " ptr=" << x.second.block << "),";
}
return ss.str();
}
size_t Join::TSRowRef::size() const
{
return ts.size();
}
std::optional<std::pair<Join::ASOFTimeType, Join::RowRef>> Join::TSRowRef::findAsof(Join::ASOFTimeType t) const
{
auto it = ts.upper_bound(t);
if (it == ts.cbegin())
return {};
return *(--it);
}
namespace
{
/// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN.
template <ASTTableJoin::Strictness STRICTNESS, typename Map, typename KeyGetter>
struct Inserter
{
static void insert(Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool);
static void insert(const Join &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool);
};
template <typename Map, typename KeyGetter>
struct Inserter<ASTTableJoin::Strictness::Any, Map, KeyGetter>
{
static ALWAYS_INLINE void insert(Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool)
static ALWAYS_INLINE void insert(const Join &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool)
{
auto emplace_result = key_getter.emplaceKey(map, i, pool);
@ -409,7 +389,7 @@ namespace
template <typename Map, typename KeyGetter>
struct Inserter<ASTTableJoin::Strictness::All, Map, KeyGetter>
{
static ALWAYS_INLINE void insert(Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool)
static ALWAYS_INLINE void insert(const Join &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool)
{
auto emplace_result = key_getter.emplaceKey(map, i, pool);
@ -435,26 +415,22 @@ namespace
template <typename Map, typename KeyGetter>
struct Inserter<ASTTableJoin::Strictness::Asof, Map, KeyGetter>
{
template<typename AsofGetter>
static ALWAYS_INLINE void insert(Map & map, KeyGetter & key_getter, AsofGetter & asof_getter, Block * stored_block, size_t i, Arena & pool)
static ALWAYS_INLINE void insert(Join & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool,
const IColumn * asof_column)
{
auto emplace_result = key_getter.emplaceKey(map, i, pool);
typename Map::mapped_type * time_series_map = &emplace_result.getMapped();
if (emplace_result.isInserted())
{
time_series_map = new (time_series_map) typename Map::mapped_type();
}
auto k = asof_getter.getKey(i, pool);
time_series_map->insert(k, stored_block, i);
// std::cout << "inserted key into time series map=" << k << " result=" << time_series_map->dumpStructure() << std::endl;
time_series_map = new (time_series_map) typename Map::mapped_type(join.getAsofType());
time_series_map->insert(join.getAsofType(), asof_column, stored_block, i);
}
};
template <ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map, bool has_null_map>
void NO_INLINE insertFromBlockImplTypeCase(
Map & map, size_t rows, const ColumnRawPtrs & key_columns,
Join & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns,
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool)
{
const IColumn * asof_column [[maybe_unused]] = nullptr;
@ -469,30 +445,28 @@ namespace
continue;
if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof)
{
auto asof_getter = Join::AsofGetterType(asof_column);
Inserter<STRICTNESS, Map, KeyGetter>::insert(map, key_getter, asof_getter, stored_block, i, pool);
} else
Inserter<STRICTNESS, Map, KeyGetter>::insert(map, key_getter, stored_block, i, pool);
Inserter<STRICTNESS, Map, KeyGetter>::insert(join, map, key_getter, stored_block, i, pool, asof_column);
else
Inserter<STRICTNESS, Map, KeyGetter>::insert(join, map, key_getter, stored_block, i, pool);
}
}
template <ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map>
void insertFromBlockImplType(
Map & map, size_t rows, const ColumnRawPtrs & key_columns,
Join & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns,
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool)
{
if (null_map)
insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, true>(map, rows, key_columns, key_sizes, stored_block, null_map, pool);
insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, true>(join, map, rows, key_columns, key_sizes, stored_block, null_map, pool);
else
insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, false>(map, rows, key_columns, key_sizes, stored_block, null_map, pool);
insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, false>(join, map, rows, key_columns, key_sizes, stored_block, null_map, pool);
}
template <ASTTableJoin::Strictness STRICTNESS, typename Maps>
void insertFromBlockImpl(
Join::Type type, Maps & maps, size_t rows, const ColumnRawPtrs & key_columns,
Join & join, Join::Type type, Maps & maps, size_t rows, const ColumnRawPtrs & key_columns,
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool)
{
switch (type)
@ -503,7 +477,7 @@ namespace
#define M(TYPE) \
case Join::Type::TYPE: \
insertFromBlockImplType<STRICTNESS, typename KeyGetterForType<Join::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>(\
*maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, pool); \
join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, pool); \
break;
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
@ -511,10 +485,44 @@ namespace
}
}
void Join::prepareBlockListStructure(Block & stored_block)
{
if (isRightOrFull(kind))
{
/** Move the key columns to the beginning of the block.
* This is where NonJoinedBlockInputStream will expect.
*/
size_t key_num = 0;
for (const auto & name : key_names_right)
{
size_t pos = stored_block.getPositionByName(name);
ColumnWithTypeAndName col = stored_block.safeGetByPosition(pos);
stored_block.erase(pos);
stored_block.insert(key_num, std::move(col));
++key_num;
}
}
else
{
NameSet erased; /// HOTFIX: there could be duplicates in JOIN ON section
/// Remove the key columns from stored_block, as they are not needed.
/// However, do not erase the ASOF column if this is an asof join
for (const auto & name : key_names_right)
{
if (strictness == ASTTableJoin::Strictness::Asof && name == key_names_right.back())
break; // this is the last column so break is OK
if (!erased.count(name))
stored_block.erase(stored_block.getPositionByName(name));
erased.insert(name);
}
}
}
bool Join::insertFromBlock(const Block & block)
{
std::unique_lock lock(rwlock);
LOG_DEBUG(log, "joinBlock: " << block.dumpStructure());
if (empty())
throw Exception("Logical error: Join was not initialized", ErrorCodes::LOGICAL_ERROR);
@ -543,33 +551,7 @@ bool Join::insertFromBlock(const Block & block)
blocks.push_back(block);
Block * stored_block = &blocks.back();
if (isRightOrFull(kind))
{
/** Move the key columns to the beginning of the block.
* This is where NonJoinedBlockInputStream will expect.
*/
size_t key_num = 0;
for (const auto & name : key_names_right)
{
size_t pos = stored_block->getPositionByName(name);
ColumnWithTypeAndName col = stored_block->safeGetByPosition(pos);
stored_block->erase(pos);
stored_block->insert(key_num, std::move(col));
++key_num;
}
}
else
{
NameSet erased; /// HOTFIX: there could be duplicates in JOIN ON section
/// Remove the key columns from stored_block, as they are not needed.
for (const auto & name : key_names_right)
{
if (!erased.count(name))
stored_block->erase(stored_block->getPositionByName(name));
erased.insert(name);
}
}
prepareBlockListStructure(*stored_block);
size_t size = stored_block->columns();
@ -590,7 +572,7 @@ bool Join::insertFromBlock(const Block & block)
{
dispatch([&](auto, auto strictness_, auto & map)
{
insertFromBlockImpl<strictness_>(type, map, rows, key_columns, key_sizes, stored_block, null_map, pool);
insertFromBlockImpl<strictness_>(*this, type, map, rows, key_columns, key_sizes, stored_block, null_map, pool);
});
}
@ -608,7 +590,9 @@ public:
AddedColumns(const Block & sample_block_with_columns_to_add,
const Block & block_with_columns_to_add,
const Block & block, size_t num_columns_to_skip)
const Block & block,
const Block & blocklist_sample,
const ColumnsWithTypeAndName & extras)
{
size_t num_columns_to_add = sample_block_with_columns_to_add.columns();
@ -622,8 +606,14 @@ public:
/// Don't insert column if it's in left block or not explicitly required.
if (!block.has(src_column.name) && block_with_columns_to_add.has(src_column.name))
addColumn(src_column, num_columns_to_skip + i);
addColumn(src_column);
}
for (auto & extra : extras)
addColumn(extra);
for (auto & tn : type_name)
right_indexes.push_back(blocklist_sample.getPositionByName(tn.second));
}
size_t size() const { return columns.size(); }
@ -651,12 +641,11 @@ private:
MutableColumns columns;
std::vector<size_t> right_indexes;
void addColumn(const ColumnWithTypeAndName & src_column, size_t idx)
void addColumn(const ColumnWithTypeAndName & src_column)
{
columns.push_back(src_column.column->cloneEmpty());
columns.back()->reserve(src_column.column->size());
type_name.emplace_back(src_column.type, src_column.name);
right_indexes.push_back(idx);
}
};
@ -678,20 +667,6 @@ void addFoundRow(const typename Map::mapped_type & mapped, AddedColumns & added,
}
};
template <typename Map>
bool addFoundRowAsof(const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset [[maybe_unused]], Join::ASOFTimeType asof_key)
{
if (auto v = mapped.findAsof(asof_key))
{
std::pair<Join::ASOFTimeType, Join::RowRef> res = *v;
// std::cout << "Adder::addFound" << " to_add" << num_columns_to_add << " i=" << i << " asof_key=" << asof_key << " found=" << res.first << std::endl;
added.appendFromBlock(*res.second.block, res.second.row_num);
return true;
}
// std::cout << "Adder::addFound" << " not found in map" << num_columns_to_add << " i=" << i << " asof_key=" << asof_key << std::endl;
return false;
}
template <bool _add_missing>
void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & current_offset [[maybe_unused]])
{
@ -707,7 +682,7 @@ void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & cur
/// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS).
template <bool _add_missing, ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map, bool _has_null_map>
std::unique_ptr<IColumn::Offsets> NO_INLINE joinRightIndexedColumns(
const Map & map, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes,
const Join & join, const Map & map, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes,
AddedColumns & added_columns, ConstNullMapPtr null_map, IColumn::Filter & filter)
{
std::unique_ptr<IColumn::Offsets> offsets_to_replicate;
@ -740,14 +715,11 @@ std::unique_ptr<IColumn::Offsets> NO_INLINE joinRightIndexedColumns(
if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof)
{
Join::AsofGetterType asof_getter(asof_column);
auto asof_key = asof_getter.getKey(i, pool);
bool actually_found = addFoundRowAsof<Map>(mapped, added_columns, current_offset, asof_key);
if (actually_found)
if (const RowRef * found = mapped.findAsof(join.getAsofType(), asof_column, i))
{
filter[i] = 1;
mapped.setUsed();
added_columns.appendFromBlock(*found->block, found->row_num);
}
else
addNotFoundRow<_add_missing>(added_columns, current_offset);
@ -772,7 +744,7 @@ std::unique_ptr<IColumn::Offsets> NO_INLINE joinRightIndexedColumns(
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map>
IColumn::Filter joinRightColumns(
const Map & map, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes,
const Join & join, const Map & map, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes,
AddedColumns & added_columns, ConstNullMapPtr null_map, std::unique_ptr<IColumn::Offsets> & offsets_to_replicate)
{
constexpr bool left_or_full = static_in_v<KIND, ASTTableJoin::Kind::Left, ASTTableJoin::Kind::Full>;
@ -781,17 +753,17 @@ IColumn::Filter joinRightColumns(
if (null_map)
offsets_to_replicate = joinRightIndexedColumns<left_or_full, STRICTNESS, KeyGetter, Map, true>(
map, rows, key_columns, key_sizes, added_columns, null_map, filter);
join, map, rows, key_columns, key_sizes, added_columns, null_map, filter);
else
offsets_to_replicate = joinRightIndexedColumns<left_or_full, STRICTNESS, KeyGetter, Map, false>(
map, rows, key_columns, key_sizes, added_columns, null_map, filter);
join, map, rows, key_columns, key_sizes, added_columns, null_map, filter);
return filter;
}
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Maps>
IColumn::Filter switchJoinRightColumns(
Join::Type type,
Join::Type type, const Join & join,
const Maps & maps_, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes,
AddedColumns & added_columns, ConstNullMapPtr null_map,
std::unique_ptr<IColumn::Offsets> & offsets_to_replicate)
@ -801,7 +773,7 @@ IColumn::Filter switchJoinRightColumns(
#define M(TYPE) \
case Join::Type::TYPE: \
return joinRightColumns<KIND, STRICTNESS, typename KeyGetterForType<Join::Type::TYPE, const std::remove_reference_t<decltype(*maps_.TYPE)>>::Type>(\
*maps_.TYPE, rows, key_columns, key_sizes, added_columns, null_map, offsets_to_replicate);
join, *maps_.TYPE, rows, key_columns, key_sizes, added_columns, null_map, offsets_to_replicate);
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
@ -865,25 +837,22 @@ void Join::joinBlockImpl(
/** For LEFT/INNER JOIN, the saved blocks do not contain keys.
* For FULL/RIGHT JOIN, the saved blocks contain keys;
* but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped.
* For ASOF, the last column is used as the ASOF column
*/
size_t num_columns_to_skip = 0;
if constexpr (right_or_full)
num_columns_to_skip = keys_size;
/// Add new columns to the block.
AddedColumns added(sample_block_with_columns_to_add, block_with_columns_to_add, block, num_columns_to_skip);
ColumnsWithTypeAndName extras;
if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof)
extras.push_back(sample_block_with_keys.getByName(key_names_right.back()));
AddedColumns added(sample_block_with_columns_to_add, block_with_columns_to_add, block, blocklist_sample, extras);
std::unique_ptr<IColumn::Offsets> offsets_to_replicate;
IColumn::Filter row_filter = switchJoinRightColumns<KIND, STRICTNESS>(
type, maps_, block.rows(), key_columns, key_sizes, added, null_map, offsets_to_replicate);
type, *this, maps_, block.rows(), key_columns, key_sizes, added, null_map, offsets_to_replicate);
for (size_t i = 0; i < added.size(); ++i)
block.insert(added.moveColumn(i));
/// Filter & insert missing rows
auto right_keys = requiredRightKeys(key_names_right, columns_added_by_join);
if constexpr (STRICTNESS == ASTTableJoin::Strictness::Any || STRICTNESS == ASTTableJoin::Strictness::Asof)
@ -1122,7 +1091,6 @@ void Join::joinGet(Block & block, const String & column_name) const
void Join::joinBlock(Block & block, const Names & key_names_left, const NamesAndTypesList & columns_added_by_join) const
{
std::shared_lock lock(rwlock);
LOG_DEBUG(log, "joinBlock: " << block.dumpStructure());
checkTypesOfKeys(block, key_names_left, sample_block_with_keys);

View File

@ -6,6 +6,7 @@
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/AggregationCommon.h>
#include <Interpreters/RowRefs.h>
#include <Core/SettingsCommon.h>
#include <Common/Arena.h>
@ -130,42 +131,7 @@ public:
size_t getTotalByteCount() const;
ASTTableJoin::Kind getKind() const { return kind; }
/// Reference to the row in block.
struct RowRef
{
const Block * block = nullptr;
size_t row_num = 0;
RowRef() {}
RowRef(const Block * block_, size_t row_num_) : block(block_), row_num(row_num_) {}
};
/// Single linked list of references to rows. Used for ALL JOINs (non-unique JOINs)
struct RowRefList : RowRef
{
RowRefList * next = nullptr;
RowRefList() {}
RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_) {}
};
/// Map for a time series
using ASOFTimeType = UInt32;
using AsofGetterType = ColumnsHashing::HashMethodOneNumber<ASOFTimeType, ASOFTimeType, ASOFTimeType, false>;
struct TSRowRef
{
// TODO use the arena allocator to get memory for this
// This would require ditching std::map because std::allocator is incompatible with the arena allocator
std::map<ASOFTimeType, RowRef> ts;
TSRowRef() {}
void insert(ASOFTimeType t, const Block * block, size_t row_num);
std::optional<std::pair<ASOFTimeType, RowRef>> findAsof(ASOFTimeType t) const;
std::string dumpStructure() const;
size_t size() const;
};
AsofRowRefs::Type getAsofType() const { return *asof_type; }
/** Depending on template parameter, adds or doesn't add a flag, that element was used (row was joined).
* Depending on template parameter, decide whether to overwrite existing values when encountering the same key again
@ -297,7 +263,7 @@ public:
using MapsAnyFull = MapsTemplate<WithFlags<true, false, RowRef>>;
using MapsAnyFullOverwrite = MapsTemplate<WithFlags<true, true, RowRef>>;
using MapsAllFull = MapsTemplate<WithFlags<true, false, RowRefList>>;
using MapsAsof = MapsTemplate<WithFlags<false, false, TSRowRef>>;
using MapsAsof = MapsTemplate<WithFlags<false, false, AsofRowRefs>>;
template <ASTTableJoin::Kind KIND>
struct KindTrait
@ -400,6 +366,7 @@ private:
private:
Type type = Type::EMPTY;
std::optional<AsofRowRefs::Type> asof_type;
static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes);
@ -410,6 +377,9 @@ private:
/// Block with key columns in the same order they appear in the right-side table.
Block sample_block_with_keys;
/// Block as it would appear in the BlockList
Block blocklist_sample;
Poco::Logger * log;
/// Limits for maximum map size.
@ -426,6 +396,11 @@ private:
void init(Type type_);
/** Take an inserted block and discard everything that does not need to be stored
* Example, remove the keys as they come from the LHS block, but do keep the ASOF timestamps
*/
void prepareBlockListStructure(Block & stored_block);
/// Throw an exception if blocks have different types of key columns.
void checkTypesOfKeys(const Block & block_left, const Names & key_names_left, const Block & block_right) const;

View File

@ -1,8 +1,10 @@
#include <Common/typeid_cast.h>
#include <Core/NamesAndTypes.h>
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/AsteriskSemantic.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
@ -22,11 +24,122 @@ namespace ErrorCodes
extern const int TOO_DEEP_AST;
extern const int AMBIGUOUS_COLUMN_NAME;
extern const int NOT_IMPLEMENTED;
extern const int UNKNOWN_IDENTIFIER;
}
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context);
namespace
{
/// Replace asterisks in select_expression_list with column identifiers
class ExtractAsterisksMatcher
{
public:
using Visitor = InDepthNodeVisitor<ExtractAsterisksMatcher, true>;
struct Data
{
std::unordered_map<String, NamesAndTypesList> table_columns;
std::vector<String> tables_order;
std::shared_ptr<ASTExpressionList> new_select_expression_list;
Data(const Context & context, const std::vector<const ASTTableExpression *> & table_expressions)
{
tables_order.reserve(table_expressions.size());
for (const auto & expr : table_expressions)
{
if (expr->subquery)
{
table_columns.clear();
tables_order.clear();
break;
}
String table_name = DatabaseAndTableWithAlias(*expr, context.getCurrentDatabase()).getQualifiedNamePrefix(false);
NamesAndTypesList columns = getNamesAndTypeListFromTableExpression(*expr, context);
tables_order.push_back(table_name);
table_columns.emplace(std::move(table_name), std::move(columns));
}
}
void addTableColumns(const String & table_name)
{
auto it = table_columns.find(table_name);
if (it == table_columns.end())
throw Exception("Unknown qualified identifier: " + table_name, ErrorCodes::UNKNOWN_IDENTIFIER);
for (const auto & column : it->second)
new_select_expression_list->children.push_back(
std::make_shared<ASTIdentifier>(std::vector<String>{it->first, column.name}));
}
};
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return false; }
static void visit(ASTPtr & ast, Data & data)
{
if (auto * t = ast->as<ASTSelectQuery>())
visit(*t, ast, data);
if (auto * t = ast->as<ASTExpressionList>())
visit(*t, ast, data);
}
private:
static void visit(ASTSelectQuery & node, ASTPtr &, Data & data)
{
if (data.table_columns.empty())
return;
Visitor(data).visit(node.select_expression_list);
if (!data.new_select_expression_list)
return;
size_t pos = 0;
for (; pos < node.children.size(); ++pos)
if (node.children[pos].get() == node.select_expression_list.get())
break;
if (pos == node.children.size())
throw Exception("No select expressions list in select", ErrorCodes::NOT_IMPLEMENTED);
node.select_expression_list = data.new_select_expression_list;
node.children[pos] = node.select_expression_list;
}
static void visit(ASTExpressionList & node, ASTPtr &, Data & data)
{
bool has_asterisks = false;
data.new_select_expression_list = std::make_shared<ASTExpressionList>();
data.new_select_expression_list->children.reserve(node.children.size());
for (auto & child : node.children)
{
if (child->as<ASTAsterisk>())
{
has_asterisks = true;
for (auto & table_name : data.tables_order)
data.addTableColumns(table_name);
}
else if (child->as<ASTQualifiedAsterisk>())
{
has_asterisks = true;
if (child->children.size() != 1)
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
ASTIdentifier & identifier = child->children[0]->as<ASTIdentifier &>();
data.addTableColumns(identifier.name);
}
else
data.new_select_expression_list->children.push_back(child);
}
if (!has_asterisks)
data.new_select_expression_list.reset();
}
};
/// Find columns with aliases to push them into rewritten subselects.
/// Normalize table aliases: table_name.column_name -> table_alias.column_name
/// Make aliases maps (alias -> column_name, column_name -> alias)
@ -41,7 +154,7 @@ struct ColumnAliasesMatcher
std::vector<std::pair<ASTIdentifier *, bool>> compound_identifiers;
std::set<String> allowed_long_names; /// original names allowed as aliases '--t.x as t.x' (select expressions only).
Data(std::vector<DatabaseAndTableWithAlias> && tables_)
Data(const std::vector<DatabaseAndTableWithAlias> && tables_)
: tables(tables_)
, public_names(false)
{}
@ -101,7 +214,7 @@ struct ColumnAliasesMatcher
visit(*t, ast, data);
if (ast->as<ASTAsterisk>() || ast->as<ASTQualifiedAsterisk>())
throw Exception("Multiple JOIN do not support asterisks yet", ErrorCodes::NOT_IMPLEMENTED);
throw Exception("Multiple JOIN do not support asterisks for complex queries yet", ErrorCodes::NOT_IMPLEMENTED);
}
static void visit(ASTIdentifier & node, ASTPtr &, Data & data)
@ -190,7 +303,7 @@ struct RewriteTablesVisitorData
}
};
bool needRewrite(ASTSelectQuery & select)
bool needRewrite(ASTSelectQuery & select, std::vector<const ASTTableExpression *> & table_expressions)
{
if (!select.tables)
return false;
@ -203,9 +316,16 @@ bool needRewrite(ASTSelectQuery & select)
if (num_tables <= 2)
return false;
for (size_t i = 1; i < tables->children.size(); ++i)
table_expressions.reserve(num_tables);
for (size_t i = 0; i < num_tables; ++i)
{
const auto * table = tables->children[i]->as<ASTTablesInSelectQueryElement>();
if (table && table->table_expression)
if (const auto * expression = table->table_expression->as<ASTTableExpression>())
table_expressions.push_back(expression);
if (!i)
continue;
if (!table || !table->table_join)
throw Exception("Multiple JOIN expects joined tables", ErrorCodes::LOGICAL_ERROR);
@ -223,6 +343,7 @@ bool needRewrite(ASTSelectQuery & select)
using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>;
using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
using ExtractAsterisksVisitor = ExtractAsterisksMatcher::Visitor;
using ColumnAliasesVisitor = InDepthNodeVisitor<ColumnAliasesMatcher, true>;
using AppendSemanticMatcher = OneTypeMatcher<AppendSemanticVisitorData>;
using AppendSemanticVisitor = InDepthNodeVisitor<AppendSemanticMatcher, true>;
@ -236,13 +357,17 @@ void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data)
visit(*t, ast, data);
}
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & data)
void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data)
{
using RevertedAliases = AsteriskSemantic::RevertedAliases;
if (!needRewrite(select))
std::vector<const ASTTableExpression *> table_expressions;
if (!needRewrite(select, table_expressions))
return;
ExtractAsterisksVisitor::Data asterisks_data(data.context, table_expressions);
ExtractAsterisksVisitor(asterisks_data).visit(ast);
ColumnAliasesVisitor::Data aliases_data(getDatabaseAndTables(select, ""));
if (select.select_expression_list)
{

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