Merge branch 'master' into issue-3578

This commit is contained in:
chertus 2018-07-13 14:30:42 +03:00
commit c8f71cb539
218 changed files with 2647 additions and 1312 deletions

6
.gitignore vendored
View File

@ -10,10 +10,8 @@
*.logrt
/build
/docs/en_single_page/
/docs/ru_single_page/
/docs/venv/
/docs/build/
/docs/build
/docs/tools/venv/
/docs/en/development/build/
/docs/ru/development/build/

View File

@ -1,4 +1,15 @@
# en:
## Improvements:
* Added Nullable support for runningDifference function. [#2590](https://github.com/yandex/ClickHouse/issues/2590)
## Bug fiexs:
* Fixed switching to default databses in case of client reconection. [#2580](https://github.com/yandex/ClickHouse/issues/2580)
# ru:
## Улучшения:
* Добавлена поддержка Nullable для функции runningDifference. [#2590](https://github.com/yandex/ClickHouse/issues/2590)
## Исправление ошибок:
* Исправлено переключение на дефолтную базу данных при переподключении клиента. [#2580](https://github.com/yandex/ClickHouse/issues/2580)

View File

@ -1,3 +1,78 @@
# ClickHouse release 1.1.54388, 2018-06-28
## New features:
* Support for the `ALTER TABLE t DELETE WHERE` query for replicated tables. Added the `system.mutations` table to track progress of this type of queries.
* Support for the `ALTER TABLE t [REPLACE|ATTACH] PARTITION` query for MergeTree tables.
* Support for the `TRUNCATE TABLE` query ([Winter Zhang](https://github.com/yandex/ClickHouse/pull/2260)).
* Several new `SYSTEM` queries for replicated tables (`RESTART REPLICAS`, `SYNC REPLICA`, `[STOP|START] [MERGES|FETCHES|SENDS REPLICATED|REPLICATION QUEUES]`).
* Added the ability to write to a table with the MySQL engine and the corresponding table function ([sundy-li](https://github.com/yandex/ClickHouse/pull/2294)).
* Added the `url()` table function and the `URL` table engine ([Alexander Sapin](https://github.com/yandex/ClickHouse/pull/2501)).
* Added the `windowFunnel` aggregate function ([sundy-li](https://github.com/yandex/ClickHouse/pull/2352)).
* New `startsWith` and `endsWith` functions for strings ([Vadim Plakhtinsky](https://github.com/yandex/ClickHouse/pull/2429)).
* The `numbers()` table function now allows you to specify the offset ([Winter Zhang](https://github.com/yandex/ClickHouse/pull/2535)).
* The password to `clickhouse-client` can be entered interactively.
* Server logs can now be sent to syslog ([Alexander Krasheninnikov](https://github.com/yandex/ClickHouse/pull/2459)).
* Support for logging in dictionaries with a shared library source ([Alexander Sapin](https://github.com/yandex/ClickHouse/pull/2472)).
* Support for custom CSV delimiters ([Ivan Zhukov](https://github.com/yandex/ClickHouse/pull/2263)).
* Added the `date_time_input_format` setting. If you switch this setting to `'best_effort'`, DateTime values will be read in a wide range of formats.
* Added the `clickhouse-obfuscator` utility for data obfuscation. Usage example: publishing data used in performance tests.
## Experimental features:
* Added the ability to calculate `and` arguments only where they are needed ([Anastasia Tsarkova](https://github.com/yandex/ClickHouse/pull/2272)).
* JIT compilation to native code is now available for some expressions ([pyos](https://github.com/yandex/ClickHouse/pull/2277)).
## Bug fixes:
* Duplicates no longer appear for a query with `DISTINCT` and `ORDER BY`.
* Queries with `ARRAY JOIN` and `arrayFilter` no longer return an incorrect result.
* Fixed an error when reading an array column from a Nested structure ([#2066](https://github.com/yandex/ClickHouse/issues/2066)).
* Fixed an error when analyzing queries with a HAVING section like `HAVING tuple IN (...)`.
* Fixed an error when analyzing queries with recursive aliases.
* Fixed an error when reading from ReplacingMergeTree with a condition in PREWHERE that filters all rows ([#2525](https://github.com/yandex/ClickHouse/issues/2525)).
* User profile settings were not applied when using sessions in the HTTP interface.
* Fixed how settings are applied from the command line parameters in `clickhouse-local`.
* The ZooKeeper client library now uses the session timeout received from the server.
* Fixed a bug in the ZooKeeper client library when the client waited for the server response longer than the timeout.
* Fixed pruning of parts for queries with conditions on partition key columns ([#2342](https://github.com/yandex/ClickHouse/issues/2342)).
* Merges are now possible after `CLEAR COLUMN IN PARTITION` ([#2315](https://github.com/yandex/ClickHouse/issues/2315)).
* Type mapping in the ODBC table function has been fixed ([sundy-li](https://github.com/yandex/ClickHouse/pull/2268)).
* Type comparisons have been fixed for `DateTime` with and without the time zone ([Alexander Bocharov](https://github.com/yandex/ClickHouse/pull/2400)).
* Fixed syntactic parsing and formatting of the `CAST` operator.
* Fixed insertion into a materialized view for the Distributed table engine ([Babacar Diassé](https://github.com/yandex/ClickHouse/pull/2411)).
* Fixed a race condition when writing data from the `Kafka` engine to materialized views ([Yangkuan Liu](https://github.com/yandex/ClickHouse/pull/2448)).
* Fixed SSRF in the `remote()` table function.
* Fixed exit behavior of `clickhouse-client` in multiline mode ([#2510](https://github.com/yandex/ClickHouse/issues/2510)).
## Improvements:
* Background tasks in replicated tables are now performed in a thread pool instead of in separate threads ([Silviu Caragea](https://github.com/yandex/ClickHouse/pull/1722)).
* Improved LZ4 compression performance.
* Faster analysis for queries with a large number of JOINs and sub-queries.
* The DNS cache is now updated automatically when there are too many network errors.
* Table inserts no longer occur if the insert into one of the materialized views is not possible because it has too many parts.
* Corrected the discrepancy in the event counters `Query`, `SelectQuery`, and `InsertQuery`.
* Expressions like `tuple IN (SELECT tuple)` are allowed if the tuple types match.
* A server with replicated tables can start even if you haven't configured ZooKeeper.
* When calculating the number of available CPU cores, limits on cgroups are now taken into account ([Atri Sharma](https://github.com/yandex/ClickHouse/pull/2325)).
* Added chown for config directories in the systemd config file ([Mikhail Shiryaev](https://github.com/yandex/ClickHouse/pull/2421)).
## Build changes:
* The gcc8 compiler can be used for builds.
* Added the ability to build llvm from a submodule.
* The version of the librdkafka library has been updated to v0.11.4.
* Added the ability to use the system libcpuid library. The library version has been updated to 0.4.0.
* Fixed the build using the vectorclass library ([Babacar Diassé](https://github.com/yandex/ClickHouse/pull/2274)).
* Cmake now generates files for ninja by default (like when using `-G Ninja`).
* Added the ability to use the libtinfo library instead of libtermcap ([Georgy Kondratiev](https://github.com/yandex/ClickHouse/pull/2519)).
* Fixed a header file conflict in Fedora Rawhide ([#2520](https://github.com/yandex/ClickHouse/issues/2520)).
## Backward incompatible changes:
* Removed escaping in `Vertical` and `Pretty*` formats and deleted the `VerticalRaw` format.
# ClickHouse release 1.1.54385, 2018-06-01
## Bug fixes:

View File

@ -43,7 +43,8 @@ include (cmake/arch.cmake)
if (CMAKE_GENERATOR STREQUAL "Ninja")
# Turn on colored output. https://github.com/ninja-build/ninja/wiki/FAQ
set (COMPILER_FLAGS "${COMPILER_FLAGS} -fdiagnostics-color=always")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fdiagnostics-color=always")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-color=always")
endif ()
if (NOT MSVC)

View File

@ -1,6 +1,13 @@
# ClickHouse
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
Learn more about ClickHouse at [https://clickhouse.yandex/](https://clickhouse.yandex/)
## Useful links
* [Official website](https://clickhouse.yandex/) has quick high-level overview of ClickHouse on main page.
* [Tutorial](https://clickhouse.yandex/tutorial.html) shows how to set up and query small ClickHouse cluster.
* [Documentation](https://clickhouse.yandex/docs/en/) provides more in-depth information.
* [Contacts](https://clickhouse.yandex/#contacts) can help to get your questions answered if there are any.
[![Build Status](https://travis-ci.org/yandex/ClickHouse.svg?branch=master)](https://travis-ci.org/yandex/ClickHouse)

View File

@ -1,4 +1,6 @@
option (USE_INTERNAL_CPUID_LIBRARY "Set to FALSE to use system cpuid library instead of bundled" ${NOT_UNBUNDLED})
if (NOT ARCH_ARM)
option (USE_INTERNAL_CPUID_LIBRARY "Set to FALSE to use system cpuid library instead of bundled" ${NOT_UNBUNDLED})
endif ()
#if (USE_INTERNAL_CPUID_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcpuid/include/cpuid/libcpuid.h")
# message (WARNING "submodule contrib/libcpuid is missing. to fix try run: \n git submodule update --init --recursive")

View File

@ -75,7 +75,7 @@ if (ENABLE_TCMALLOC AND USE_INTERNAL_GPERFTOOLS_LIBRARY)
add_subdirectory (libtcmalloc)
endif ()
if (NOT ARCH_ARM)
if (USE_INTERNAL_CPUID_LIBRARY)
add_subdirectory (libcpuid)
endif ()

View File

@ -1,7 +1,7 @@
# This strings autochanged from release_lib.sh:
set(VERSION_DESCRIBE v1.1.54388-testing)
set(VERSION_REVISION 54388)
set(VERSION_GITHASH 2447755700f40af317cb80ba8800b94d6350d148)
set(VERSION_DESCRIBE v1.1.54394-testing)
set(VERSION_REVISION 54394)
set(VERSION_GITHASH 875ea0f4eaa3592f1fe628b6a1150d91b04ad574)
# end of autochange
set (VERSION_MAJOR 1)

View File

@ -376,7 +376,6 @@ private:
echo_queries = config().getBool("echo", false);
}
connection_parameters = ConnectionParameters(config());
connect();
/// Initialize DateLUT here to avoid counting time spent here as query execution time.
@ -494,6 +493,8 @@ private:
void connect()
{
connection_parameters = ConnectionParameters(config());
if (is_interactive)
std::cout << "Connecting to "
<< (!connection_parameters.default_database.empty() ? "database " + connection_parameters.default_database + " at " : "")

View File

@ -15,17 +15,18 @@
</openSSL>
<!--
It's a custom prompt settings for the clickhouse-client
Possible macros:
Possible macros:
{host}
{port}
{user}
{database}
{database}
{display_name}
Terminal colors: https://misc.flogisoft.com/bash/tip_colors_and_formatting
See also: https://wiki.hackzine.org/development/misc/readline-color-prompt.html
-->
<prompt_by_server_display_name>
<default>{display_name} :) </default>
<test>{display_name} \e[1;32m:)\e[0m </test> <!-- if it matched to the substring "test" in the server display name - -->
<production>{display_name} \e[1;31m:)\e[0m </production> <!-- if it matched to the substring "production" in the server display name -->
<test>{display_name} \x01\e[1;32m\x02:)\x01\e[0m\x02 </test> <!-- if it matched to the substring "test" in the server display name - -->
<production>{display_name} \x01\e[1;31m\x02:)\x01\e[0m\x02 </production> <!-- if it matched to the substring "production" in the server display name -->
</prompt_by_server_display_name>
</config>

View File

@ -3,6 +3,7 @@
#include <memory>
#include <sys/resource.h>
#include <errno.h>
#include <Poco/Version.h>
#include <Poco/DirectoryIterator.h>
#include <Poco/Net/HTTPServer.h>
#include <Poco/Net/NetException.h>
@ -341,7 +342,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024));
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
http_params->setTimeout(settings.receive_timeout);
http_params->setTimeout(settings.http_receive_timeout);
http_params->setKeepAliveTimeout(keep_alive_timeout);
std::vector<std::unique_ptr<Poco::Net::TCPServer>> servers;

View File

@ -0,0 +1,56 @@
#include <AggregateFunctions/AggregateFunctionHistogram.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <Common/FieldVisitors.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_PARAMETER;
extern const int PARAMETER_OUT_OF_BOUND;
}
namespace
{
AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, const DataTypes & arguments, const Array & params)
{
if (params.size() != 1)
throw Exception("Function " + name + " requires single parameter: bins count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (params[0].getType() != Field::Types::UInt64)
throw Exception("Invalid type for bins count", ErrorCodes::UNSUPPORTED_PARAMETER);
UInt32 bins_count = applyVisitor(FieldVisitorConvertToNumber<UInt32>(), params[0]);
auto limit = AggregateFunctionHistogramData::bins_count_limit;
if (bins_count > limit)
throw Exception("Unsupported bins count. Should not be greater than " + std::to_string(limit), ErrorCodes::PARAMETER_OUT_OF_BOUND);
if (bins_count == 0)
throw Exception("Bin count should be positive", ErrorCodes::BAD_ARGUMENTS);
assertUnary(name, arguments);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionHistogram>(*arguments[0], bins_count));
if (!res)
throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
}
void registerAggregateFunctionHistogram(AggregateFunctionFactory & factory)
{
factory.registerFunction("histogram", createAggregateFunctionHistogram);
}
}

View File

@ -0,0 +1,376 @@
#pragma once
#include <Common/Arena.h>
#include <Common/NaNUtils.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
#include <IO/VarInt.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <math.h>
#include <queue>
#include <stddef.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
extern const int INCORRECT_DATA;
}
/**
* distance compression algorigthm implementation
* http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
*/
class AggregateFunctionHistogramData
{
public:
using Mean = Float64;
using Weight = Float64;
constexpr static size_t bins_count_limit = 250;
private:
struct WeightedValue
{
Mean mean;
Weight weight;
WeightedValue operator+ (const WeightedValue& other)
{
return {mean + other.weight * (other.mean - mean) / (other.weight + weight), other.weight + weight};
}
};
private:
// quantity of stored weighted-values
UInt32 size;
// calculated lower and upper bounds of seen points
Mean lower_bound;
Mean upper_bound;
// Weighted values representation of histogram.
WeightedValue points[0];
private:
void sort()
{
std::sort(points, points + size,
[](const WeightedValue & first, const WeightedValue & second)
{
return first.mean < second.mean;
});
}
template <typename T>
struct PriorityQueueStorage
{
size_t size = 0;
T * data_ptr;
PriorityQueueStorage(T * value)
: data_ptr(value)
{
}
void push_back(T val)
{
data_ptr[size] = std::move(val);
++size;
}
void pop_back() { --size; }
T * begin() { return data_ptr; }
T * end() const { return data_ptr + size; }
bool empty() const { return size == 0; }
T & front() { return *data_ptr; }
const T & front() const { return *data_ptr; }
using value_type = T;
using reference = T&;
using const_reference = const T&;
using size_type = size_t;
};
/**
* Repeatedly fuse most close values until max_bins bins left
*/
void compress(UInt32 max_bins)
{
sort();
auto new_size = size;
if (size <= max_bins)
return;
// Maintain doubly-linked list of "active" points
// and store neighbour pairs in priority queue by distance
UInt32 previous[size + 1];
UInt32 next[size + 1];
bool active[size + 1];
std::fill(active, active + size, true);
active[size] = false;
auto delete_node = [&](UInt32 i)
{
previous[next[i]] = previous[i];
next[previous[i]] = next[i];
active[i] = false;
};
for (size_t i = 0; i <= size; ++i)
{
previous[i] = i - 1;
next[i] = i + 1;
}
next[size] = 0;
previous[0] = size;
using QueueItem = std::pair<Mean, UInt32>;
QueueItem storage[2 * size - max_bins];
std::priority_queue<
QueueItem,
PriorityQueueStorage<QueueItem>,
std::greater<QueueItem>>
queue{std::greater<QueueItem>(),
PriorityQueueStorage<QueueItem>(storage)};
auto quality = [&](UInt32 i) { return points[next[i]].mean - points[i].mean; };
for (size_t i = 0; i + 1 < size; ++i)
queue.push({quality(i), i});
while (new_size > max_bins && !queue.empty())
{
auto min_item = queue.top();
queue.pop();
auto left = min_item.second;
auto right = next[left];
if (!active[left] || !active[right] || quality(left) > min_item.first)
continue;
points[left] = points[left] + points[right];
delete_node(right);
if (active[next[left]])
queue.push({quality(left), left});
if (active[previous[left]])
queue.push({quality(previous[left]), previous[left]});
--new_size;
}
size_t left = 0;
for (size_t right = 0; right < size; ++right)
{
if (active[right])
{
points[left] = points[right];
++left;
}
}
size = new_size;
}
/***
* Delete too close points from histogram.
* Assumes that points are sorted.
*/
void unique()
{
if (size == 0)
return;
size_t left = 0;
for (auto right = left + 1; right < size; ++right)
{
// Fuse points if their text representations differ only in last digit
auto min_diff = 10 * (points[left].mean + points[right].mean) * std::numeric_limits<Mean>::epsilon();
if (points[left].mean + min_diff >= points[right].mean)
{
points[left] = points[left] + points[right];
}
else
{
++left;
points[left] = points[right];
}
}
size = left + 1;
}
public:
AggregateFunctionHistogramData()
: size(0)
, lower_bound(std::numeric_limits<Mean>::max())
, upper_bound(std::numeric_limits<Mean>::lowest())
{
static_assert(offsetof(AggregateFunctionHistogramData, points) == sizeof(AggregateFunctionHistogramData), "points should be last member");
}
static size_t structSize(size_t max_bins)
{
return sizeof(AggregateFunctionHistogramData) + max_bins * 2 * sizeof(WeightedValue);
}
void insertResultInto(ColumnVector<Mean> & to_lower, ColumnVector<Mean> & to_upper, ColumnVector<Weight> & to_weights, UInt32 max_bins)
{
compress(max_bins);
unique();
for (size_t i = 0; i < size; ++i)
{
to_lower.insert((i == 0) ? lower_bound : (points[i].mean + points[i - 1].mean) / 2);
to_upper.insert((i + 1 == size) ? upper_bound : (points[i].mean + points[i + 1].mean) / 2);
// linear density approximation
Weight lower_weight = (i == 0) ? points[i].weight : ((points[i - 1].weight) + points[i].weight * 3) / 4;
Weight upper_weight = (i + 1 == size) ? points[i].weight : (points[i + 1].weight + points[i].weight * 3) / 4;
to_weights.insert((lower_weight + upper_weight) / 2);
}
}
void add(Mean value, Weight weight, UInt32 max_bins)
{
// nans break sort and compression
// infs don't fit in bins partition method
if (!isFinite(value))
throw Exception("Invalid value (inf or nan) for aggregation by 'histogram' function", ErrorCodes::INCORRECT_DATA);
points[size] = {value, weight};
++size;
lower_bound = std::min(lower_bound, value);
upper_bound = std::max(upper_bound, value);
if (size >= max_bins * 2)
compress(max_bins);
}
void merge(const AggregateFunctionHistogramData& other, UInt32 max_bins)
{
lower_bound = std::min(lower_bound, other.lower_bound);
upper_bound = std::max(lower_bound, other.upper_bound);
for (size_t i = 0; i < other.size; i++)
{
add(other.points[i].mean, other.points[i].weight, max_bins);
}
}
void write(WriteBuffer & buf) const
{
buf.write(reinterpret_cast<const char *>(&lower_bound), sizeof(lower_bound));
buf.write(reinterpret_cast<const char *>(&upper_bound), sizeof(upper_bound));
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(points), size * sizeof(WeightedValue));
}
void read(ReadBuffer & buf, UInt32 max_bins)
{
buf.read(reinterpret_cast<char *>(&lower_bound), sizeof(lower_bound));
buf.read(reinterpret_cast<char *>(&upper_bound), sizeof(upper_bound));
readVarUInt(size, buf);
if (size > max_bins * 2)
throw Exception("Too many bins", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
buf.read(reinterpret_cast<char *>(points), size * sizeof(WeightedValue));
}
};
template <typename T>
class AggregateFunctionHistogram final: public IAggregateFunctionDataHelper<AggregateFunctionHistogramData, AggregateFunctionHistogram<T>>
{
private:
using Data = AggregateFunctionHistogramData;
const UInt32 max_bins;
public:
AggregateFunctionHistogram(UInt32 max_bins)
: max_bins(max_bins)
{
}
size_t sizeOfData() const override
{
return Data::structSize(max_bins);
}
DataTypePtr getReturnType() const override
{
DataTypes types;
auto mean = std::make_shared<DataTypeNumber<Data::Mean>>();
auto weight = std::make_shared<DataTypeNumber<Data::Weight>>();
// lower bound
types.emplace_back(mean);
// upper bound
types.emplace_back(mean);
// weight
types.emplace_back(weight);
auto tuple = std::make_shared<DataTypeTuple>(types);
return std::make_shared<DataTypeArray>(tuple);
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto val = static_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num];
this->data(place).add(static_cast<Data::Mean>(val), 1, max_bins);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs), max_bins);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
{
this->data(place).read(buf, max_bins);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
auto& data = this->data(const_cast<AggregateDataPtr>(place));
auto & to_array = static_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = to_array.getOffsets();
auto & to_tuple = static_cast<ColumnTuple &>(to_array.getData());
auto & to_lower = static_cast<ColumnVector<Data::Mean> &>(to_tuple.getColumn(0));
auto & to_upper = static_cast<ColumnVector<Data::Mean> &>(to_tuple.getColumn(1));
auto & to_weights = static_cast<ColumnVector<Data::Weight> &>(to_tuple.getColumn(2));
data.insertResultInto(to_lower, to_upper, to_weights, max_bins);
offsets_to.push_back(to_tuple.size());
}
const char * getHeaderFilePath() const override { return __FILE__; }
String getName() const override { return "histogram"; }
};
}

View File

@ -33,6 +33,7 @@ void registerAggregateFunctionCombinatorState(AggregateFunctionCombinatorFactory
void registerAggregateFunctionCombinatorMerge(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorNull(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionHistogram(AggregateFunctionFactory & factory);
void registerAggregateFunctions()
{
@ -57,6 +58,7 @@ void registerAggregateFunctions()
registerAggregateFunctionTopK(factory);
registerAggregateFunctionsBitwise(factory);
registerAggregateFunctionsMaxIntersections(factory);
registerAggregateFunctionHistogram(factory);
}
{

View File

@ -118,8 +118,8 @@ void BackgroundSchedulePool::TaskInfo::execute()
executing = false;
/// In case was scheduled while executing (including a scheduleAfter which expired) we schedule the task
/// on the queue. We don't call the function again here because this way all tasks
/// will have their chance to execute
/// on the queue. We don't call the function again here because this way all tasks
/// will have their chance to execute
if (scheduled)
pool.queue.enqueueNotification(new TaskNotification(shared_from_this()));

View File

@ -47,7 +47,7 @@ public:
KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses);
private:
size_t getFailedOpIndex(int32_t code, const Responses & responses) const;
static size_t getFailedOpIndex(int32_t code, const Responses & responses);
};
};

View File

@ -815,7 +815,7 @@ int32_t ZooKeeper::tryMultiNoThrow(const Requests & requests, Responses & respon
}
size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & responses) const
size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & responses)
{
if (responses.empty())
throw DB::Exception("Responses for multi transaction is empty", DB::ErrorCodes::LOGICAL_ERROR);
@ -833,15 +833,16 @@ size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & re
KeeperMultiException::KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses)
: KeeperException("Transaction failed at op #" + std::to_string(getFailedOpIndex(code, responses)), code),
: KeeperException("Transaction failed", code),
requests(requests), responses(responses), failed_op_index(getFailedOpIndex(code, responses))
{
addMessage("Op #" + std::to_string(failed_op_index) + ", path: " + getPathForFirstFailedOp());
}
std::string KeeperMultiException::getPathForFirstFailedOp() const
{
return requests[failed_op_index]->getPath();
}
void KeeperMultiException::check(int32_t code, const Requests & requests, const Responses & responses)

View File

@ -367,10 +367,20 @@ void read(String & s, ReadBuffer & in)
static constexpr int32_t max_string_size = 1 << 20;
int32_t size = 0;
read(size, in);
if (size < 0) /// TODO Actually it means that zookeeper node has NULL value. Maybe better to treat it like empty string.
if (size == -1)
{
/// It means that zookeeper node has NULL value. We will treat it like empty string.
s.clear();
return;
}
if (size < 0)
throw Exception("Negative size while reading string from ZooKeeper", ZooKeeper::ZMARSHALLINGERROR);
if (size > max_string_size)
throw Exception("Too large string size while reading from ZooKeeper", ZooKeeper::ZMARSHALLINGERROR);
s.resize(size);
in.read(&s[0], size);
}
@ -875,6 +885,18 @@ ZooKeeper::ResponsePtr ZooKeeper::MultiRequest::makeResponse() const { return st
ZooKeeper::ResponsePtr ZooKeeper::CloseRequest::makeResponse() const { return std::make_shared<CloseResponse>(); }
ZooKeeper::RequestPtr ZooKeeper::MultiRequest::clone() const
{
auto res = std::make_shared<MultiRequest>();
res->requests.reserve(requests.size());
for (const auto & request : requests)
res->requests.emplace_back(request->clone());
return res;
}
void ZooKeeper::CreateRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); }
void ZooKeeper::RemoveRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); }
void ZooKeeper::ExistsRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); }
@ -965,30 +987,48 @@ void ZooKeeper::receiveEvent()
if (it == operations.end())
throw Exception("Received response for unknown xid", ZRUNTIMEINCONSISTENCY);
/// After this point, we must invoke callback, that we've grabbed from 'operations'.
/// Invariant: all callbacks are invoked either in case of success or in case of error.
/// (all callbacks in 'operations' are guaranteed to be invoked)
request_info = std::move(it->second);
operations.erase(it);
CurrentMetrics::sub(CurrentMetrics::ZooKeeperRequest);
}
response = request_info.request->makeResponse();
auto elapsed_microseconds = std::chrono::duration_cast<std::chrono::microseconds>(clock::now() - request_info.time).count();
ProfileEvents::increment(ProfileEvents::ZooKeeperWaitMicroseconds, elapsed_microseconds);
}
if (err)
response->error = err;
else
try
{
response->readImpl(*in);
response->removeRootPath(root_path);
if (!response)
response = request_info.request->makeResponse();
if (err)
response->error = err;
else
{
response->readImpl(*in);
response->removeRootPath(root_path);
}
int32_t actual_length = in->count() - count_before_event;
if (length != actual_length)
throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length), ZMARSHALLINGERROR);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
response->error = ZMARSHALLINGERROR;
if (request_info.callback)
request_info.callback(*response);
throw;
}
int32_t actual_length = in->count() - count_before_event;
if (length != actual_length)
throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length), ZMARSHALLINGERROR);
/// NOTE: Exception in callback will propagate to receiveThread and will lead to session expiration. This is Ok.
/// Exception in callback will propagate to receiveThread and will lead to session expiration. This is Ok.
if (request_info.callback)
request_info.callback(*response);
@ -1507,7 +1547,11 @@ void ZooKeeper::multi(
MultiCallback callback)
{
MultiRequest request;
request.requests = requests;
/// Deep copy to avoid modifying path in presence of chroot prefix.
request.requests.reserve(requests.size());
for (const auto & elem : requests)
request.requests.emplace_back(elem->clone());
for (auto & elem : request.requests)
if (CreateRequest * create = typeid_cast<CreateRequest *>(elem.get()))

View File

@ -156,6 +156,10 @@ public:
using XID = int32_t;
using OpNum = int32_t;
struct Response;
using ResponsePtr = std::shared_ptr<Response>;
using Responses = std::vector<ResponsePtr>;
using ResponseCallback = std::function<void(const Response &)>;
struct Response
{
@ -166,9 +170,9 @@ public:
virtual void removeRootPath(const String & /* root_path */) {}
};
using ResponsePtr = std::shared_ptr<Response>;
using Responses = std::vector<ResponsePtr>;
using ResponseCallback = std::function<void(const Response &)>;
struct Request;
using RequestPtr = std::shared_ptr<Request>;
using Requests = std::vector<RequestPtr>;
struct Request
{
@ -176,6 +180,8 @@ public:
bool has_watch = false;
virtual ~Request() {}
virtual RequestPtr clone() const = 0;
virtual OpNum getOpNum() const = 0;
/// Writes length, xid, op_num, then the rest.
@ -188,11 +194,9 @@ public:
virtual String getPath() const = 0;
};
using RequestPtr = std::shared_ptr<Request>;
using Requests = std::vector<RequestPtr>;
struct HeartbeatRequest final : Request
{
RequestPtr clone() const override { return std::make_shared<HeartbeatRequest>(*this); }
OpNum getOpNum() const override { return 11; }
void writeImpl(WriteBuffer &) const override {}
ResponsePtr makeResponse() const override;
@ -222,6 +226,7 @@ public:
String scheme;
String data;
RequestPtr clone() const override { return std::make_shared<AuthRequest>(*this); }
OpNum getOpNum() const override { return 100; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
@ -235,6 +240,7 @@ public:
struct CloseRequest final : Request
{
RequestPtr clone() const override { return std::make_shared<CloseRequest>(*this); }
OpNum getOpNum() const override { return -11; }
void writeImpl(WriteBuffer &) const override {}
ResponsePtr makeResponse() const override;
@ -254,6 +260,7 @@ public:
bool is_sequential = false;
ACLs acls;
RequestPtr clone() const override { return std::make_shared<CreateRequest>(*this); }
OpNum getOpNum() const override { return 1; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
@ -274,6 +281,7 @@ public:
String path;
int32_t version = -1;
RequestPtr clone() const override { return std::make_shared<RemoveRequest>(*this); }
OpNum getOpNum() const override { return 2; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
@ -290,6 +298,7 @@ public:
{
String path;
RequestPtr clone() const override { return std::make_shared<ExistsRequest>(*this); }
OpNum getOpNum() const override { return 3; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
@ -308,6 +317,7 @@ public:
{
String path;
RequestPtr clone() const override { return std::make_shared<GetRequest>(*this); }
OpNum getOpNum() const override { return 4; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
@ -329,6 +339,7 @@ public:
String data;
int32_t version = -1;
RequestPtr clone() const override { return std::make_shared<SetRequest>(*this); }
OpNum getOpNum() const override { return 5; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
@ -347,6 +358,7 @@ public:
{
String path;
RequestPtr clone() const override { return std::make_shared<ListRequest>(*this); }
OpNum getOpNum() const override { return 12; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
@ -367,6 +379,7 @@ public:
String path;
int32_t version = -1;
RequestPtr clone() const override { return std::make_shared<CheckRequest>(*this); }
OpNum getOpNum() const override { return 13; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
@ -383,6 +396,7 @@ public:
{
Requests requests;
RequestPtr clone() const override;
OpNum getOpNum() const override { return 14; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;

View File

@ -19,13 +19,6 @@
/// The size of the I/O buffer by default.
#define DBMS_DEFAULT_BUFFER_SIZE 1048576ULL
/// When writing data, a buffer of `max_compress_block_size` size is allocated for compression. When the buffer overflows or if into the buffer
/// more or equal data is written than `min_compress_block_size`, then with the next mark, the data will also compressed
/// As a result, for small columns (numbers 1-8 bytes), with index_granularity = 8192, the block size will be 64 KB.
/// And for large columns (Title - string ~100 bytes), the block size will be ~819 KB. Due to this, the compression ratio almost does not get worse.
#define DEFAULT_MIN_COMPRESS_BLOCK_SIZE 65536
#define DEFAULT_MAX_COMPRESS_BLOCK_SIZE 1048576
/** Which blocks by default read the data (by number of rows).
* Smaller values give better cache locality, less consumption of RAM, but more overhead to process the query.
*/
@ -43,17 +36,12 @@
*/
#define DEFAULT_MERGE_BLOCK_SIZE 8192
#define DEFAULT_MAX_QUERY_SIZE 262144
#define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160)
#define DEFAULT_MAX_DISTRIBUTED_CONNECTIONS 1024
#define DEFAULT_INTERACTIVE_DELAY 100000
#define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3
/// each period reduces the error counter by 2 times
/// too short a period can cause errors to disappear immediately after creation.
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD (2 * DBMS_DEFAULT_SEND_TIMEOUT_SEC)
#define DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS 5000 /// Maximum waiting time in the request queue.
#define DBMS_DEFAULT_BACKGROUND_POOL_SIZE 16
#define DBMS_MIN_REVISION_WITH_CLIENT_INFO 54032
#define DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE 54058
@ -65,8 +53,6 @@
/// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change.
#define DBMS_TCP_PROTOCOL_VERSION 54226
#define DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS 100
/// The boundary on which the blocks for asynchronous file operations should be aligned.
#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096

View File

@ -1,20 +0,0 @@
#include <sstream>
#include <Core/SortDescription.h>
#include <Columns/Collator.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromString.h>
namespace DB
{
std::string SortColumnDescription::getID() const
{
WriteBufferFromOwnString out;
out << column_name << ", " << column_number << ", " << direction << ", " << nulls_direction;
if (collator)
out << ", collation locale: " << collator->getLocale();
return out.str();
}
}

View File

@ -26,9 +26,6 @@ struct SortColumnDescription
SortColumnDescription(const std::string & column_name_, int direction_, int nulls_direction_, const std::shared_ptr<Collator> & collator_ = nullptr)
: column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {}
/// For IBlockInputStream.
std::string getID() const;
};
/// Description of the sorting rule for several columns.

View File

@ -4,6 +4,7 @@
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnArray.h>
#include <Interpreters/evaluateMissingDefaults.h>
#include <Core/Block.h>
#include <Interpreters/evaluateMissingDefaults.h>

View File

@ -119,7 +119,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery)
if (!done_with_set)
{
if (!subquery.set->insertFromBlock(block, /*fill_set_elements=*/false))
if (!subquery.set->insertFromBlock(block))
done_with_set = true;
}

View File

@ -3,6 +3,8 @@
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Columns/ColumnAggregateFunction.h>
#include <Columns/ColumnTuple.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/FieldVisitors.h>
@ -74,7 +76,8 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream(
}
else
{
if (!column.type->isSummable())
bool is_agg_func = checkDataType<DataTypeAggregateFunction>(column.type.get());
if (!column.type->isSummable() && !is_agg_func)
{
column_numbers_not_to_aggregate.push_back(i);
continue;
@ -93,8 +96,14 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream(
{
// Create aggregator to sum this column
AggregateDescription desc;
desc.is_agg_func_type = is_agg_func;
desc.column_numbers = {i};
desc.init("sumWithOverflow", {column.type});
if (!is_agg_func)
{
desc.init("sumWithOverflow", {column.type});
}
columns_to_aggregate.emplace_back(std::move(desc));
}
else
@ -193,27 +202,34 @@ void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & me
// Do not insert if the aggregation state hasn't been created
if (desc.created)
{
try
if (desc.is_agg_func_type)
{
desc.function->insertResultInto(desc.state.data(), *desc.merged_column);
/// Update zero status of current row
if (desc.column_numbers.size() == 1)
{
// Flag row as non-empty if at least one column number if non-zero
current_row_is_zero = current_row_is_zero && desc.merged_column->get64(desc.merged_column->size() - 1) == 0;
}
else
{
/// It is sumMap aggregate function.
/// Assume that the row isn't empty in this case (just because it is compatible with previous version)
current_row_is_zero = false;
}
current_row_is_zero = false;
}
catch (...)
else
{
desc.destroyState();
throw;
try
{
desc.function->insertResultInto(desc.state.data(), *desc.merged_column);
/// Update zero status of current row
if (desc.column_numbers.size() == 1)
{
// Flag row as non-empty if at least one column number if non-zero
current_row_is_zero = current_row_is_zero && desc.merged_column->get64(desc.merged_column->size() - 1) == 0;
}
else
{
/// It is sumMap aggregate function.
/// Assume that the row isn't empty in this case (just because it is compatible with previous version)
current_row_is_zero = false;
}
}
catch (...)
{
desc.destroyState();
throw;
}
}
desc.destroyState();
}
@ -258,7 +274,7 @@ Block SummingSortedBlockInputStream::readImpl()
for (auto & desc : columns_to_aggregate)
{
// Wrap aggregated columns in a tuple to match function signature
if (checkDataType<DataTypeTuple>(desc.function->getReturnType().get()))
if (!desc.is_agg_func_type && checkDataType<DataTypeTuple>(desc.function->getReturnType().get()))
{
size_t tuple_size = desc.column_numbers.size();
MutableColumns tuple_columns(tuple_size);
@ -277,7 +293,7 @@ Block SummingSortedBlockInputStream::readImpl()
/// Place aggregation results into block.
for (auto & desc : columns_to_aggregate)
{
if (checkDataType<DataTypeTuple>(desc.function->getReturnType().get()))
if (!desc.is_agg_func_type && checkDataType<DataTypeTuple>(desc.function->getReturnType().get()))
{
/// Unpack tuple into block.
size_t tuple_size = desc.column_numbers.size();
@ -465,23 +481,32 @@ void SummingSortedBlockInputStream::addRow(SortCursor & cursor)
{
for (auto & desc : columns_to_aggregate)
{
if (!desc.created)
throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR);
// Specialized case for unary functions
if (desc.column_numbers.size() == 1)
if (desc.is_agg_func_type)
{
// desc.state is not used for AggregateFunction types
auto & col = cursor->all_columns[desc.column_numbers[0]];
desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr);
static_cast<ColumnAggregateFunction &>(*desc.merged_column).insertMergeFrom(*col, cursor->pos);
}
else
{
// Gather all source columns into a vector
ColumnRawPtrs columns(desc.column_numbers.size());
for (size_t i = 0; i < desc.column_numbers.size(); ++i)
columns[i] = cursor->all_columns[desc.column_numbers[i]];
if (!desc.created)
throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR);
desc.add_function(desc.function.get(), desc.state.data(), columns.data(), cursor->pos, nullptr);
// Specialized case for unary functions
if (desc.column_numbers.size() == 1)
{
auto & col = cursor->all_columns[desc.column_numbers[0]];
desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr);
}
else
{
// Gather all source columns into a vector
ColumnRawPtrs columns(desc.column_numbers.size());
for (size_t i = 0; i < desc.column_numbers.size(); ++i)
columns[i] = cursor->all_columns[desc.column_numbers[i]];
desc.add_function(desc.function.get(), desc.state.data(), columns.data(), cursor->pos, nullptr);
}
}
}
}

View File

@ -69,6 +69,7 @@ private:
/// Stores aggregation function, state, and columns to be used as function arguments
struct AggregateDescription
{
/// An aggregate function 'sumWithOverflow' or 'sumMap' for summing.
AggregateFunctionPtr function;
IAggregateFunction::AddFunc add_function = nullptr;
std::vector<size_t> column_numbers;
@ -76,6 +77,9 @@ private:
std::vector<char> state;
bool created = false;
/// In case when column has type AggregateFunction: use the aggregate function from itself instead of 'function' above.
bool is_agg_func_type = false;
void init(const char * function_name, const DataTypes & argument_types)
{
function = AggregateFunctionFactory::instance().get(function_name, argument_types);
@ -87,7 +91,10 @@ private:
{
if (created)
return;
function->create(state.data());
if (is_agg_func_type)
merged_column->insertDefault();
else
function->create(state.data());
created = true;
}
@ -95,7 +102,8 @@ private:
{
if (!created)
return;
function->destroy(state.data());
if (!is_agg_func_type)
function->destroy(state.data());
created = false;
}

View File

@ -241,7 +241,7 @@ void DataTypeAggregateFunction::serializeTextCSV(const IColumn & column, size_t
void DataTypeAggregateFunction::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String s;
readCSV(s, istr, settings.csv.delimiter);
readCSV(s, istr, settings.csv);
deserializeFromString(function, column, s);
}

View File

@ -415,7 +415,7 @@ void DataTypeArray::serializeTextCSV(const IColumn & column, size_t row_num, Wri
void DataTypeArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String s;
readCSV(s, istr, settings.csv.delimiter);
readCSV(s, istr, settings.csv);
ReadBufferFromString rb(s);
deserializeText(column, rb, settings);
}

View File

@ -194,7 +194,7 @@ template <typename Type>
void DataTypeEnum<Type>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
std::string name;
readCSVString(name, istr, settings.csv.delimiter);
readCSVString(name, istr, settings.csv);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name)));
}

View File

@ -197,7 +197,7 @@ void DataTypeFixedString::serializeTextCSV(const IColumn & column, size_t row_nu
void DataTypeFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
read(*this, column, [&istr, delimiter = settings.csv.delimiter](ColumnFixedString::Chars_t & data) { readCSVStringInto(data, istr, delimiter); });
read(*this, column, [&istr, &csv = settings.csv](ColumnFixedString::Chars_t & data) { readCSVStringInto(data, istr, csv); });
}

View File

@ -245,6 +245,12 @@ bool DataTypeNumberBase<T>::isValueRepresentedByInteger() const
return std::is_integral_v<T>;
}
template <typename T>
bool DataTypeNumberBase<T>::isValueRepresentedByUnsignedInteger() const
{
return std::is_integral_v<T> && std::is_unsigned_v<T>;
}
/// Explicit template instantiations - to avoid code bloat in headers.
template class DataTypeNumberBase<UInt8>;

View File

@ -46,6 +46,7 @@ public:
bool isComparable() const override { return true; }
bool isValueRepresentedByNumber() const override { return true; }
bool isValueRepresentedByInteger() const override;
bool isValueRepresentedByUnsignedInteger() const override;
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
bool haveMaximumSizeOfValue() const override { return true; }
size_t getSizeOfValueInMemory() const override { return sizeof(T); }

View File

@ -288,7 +288,7 @@ void DataTypeString::serializeTextCSV(const IColumn & column, size_t row_num, Wr
void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
read(column, [&](ColumnString::Chars_t & data) { readCSVStringInto(data, istr, settings.csv.delimiter); });
read(column, [&](ColumnString::Chars_t & data) { readCSVStringInto(data, istr, settings.csv); });
}

View File

@ -307,6 +307,10 @@ public:
*/
virtual bool isValueRepresentedByInteger() const { return false; }
/** Unsigned Integers, Date, DateTime. Not nullable.
*/
virtual bool isValueRepresentedByUnsignedInteger() const { return false; }
/** Values are unambiguously identified by contents of contiguous memory region,
* that can be obtained by IColumn::getDataAt method.
* Examples: numbers, Date, DateTime, String, FixedString,

View File

@ -83,16 +83,16 @@ static inline void skipWhitespacesAndTabs(ReadBuffer & buf)
}
static void skipRow(ReadBuffer & istr, const char delimiter, size_t num_columns)
static void skipRow(ReadBuffer & istr, const FormatSettings::CSV & settings, size_t num_columns)
{
String tmp;
for (size_t i = 0; i < num_columns; ++i)
{
skipWhitespacesAndTabs(istr);
readCSVString(tmp, istr, delimiter);
readCSVString(tmp, istr, settings);
skipWhitespacesAndTabs(istr);
skipDelimiter(istr, delimiter, i + 1 == num_columns);
skipDelimiter(istr, settings.delimiter, i + 1 == num_columns);
}
}
@ -107,7 +107,7 @@ void CSVRowInputStream::readPrefix()
String tmp;
if (with_names)
skipRow(istr, format_settings.csv.delimiter, num_columns);
skipRow(istr, format_settings.csv, num_columns);
}

View File

@ -37,6 +37,8 @@ BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & bu
FormatSettings format_settings;
format_settings.csv.delimiter = settings.format_csv_delimiter;
format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes;
format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes;
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields;
format_settings.date_time_input_format = settings.date_time_input_format;
@ -59,6 +61,8 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer &
format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers;
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;
format_settings.csv.delimiter = settings.format_csv_delimiter;
format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes;
format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes;
format_settings.pretty.max_rows = settings.output_format_pretty_max_rows;
format_settings.pretty.color = settings.output_format_pretty_color;
format_settings.write_statistics = settings.output_format_write_statistics;

View File

@ -24,6 +24,8 @@ struct FormatSettings
struct CSV
{
char delimiter = ',';
bool allow_single_quotes = true;
bool allow_double_quotes = true;
};
CSV csv;

View File

@ -204,7 +204,7 @@ private:
static bool hasNull(const PaddedPODArray<UInt8> & null_map, size_t i)
{
return null_map[i] == 1;
return null_map[i];
}
/// Both function arguments are ordinary.
@ -287,7 +287,7 @@ private:
for (size_t j = 0; j < array_size; ++j)
{
if (null_map_data[current_offset + j] == 1)
if (null_map_data[current_offset + j])
{
}
else if (compare(data[current_offset + j], value, i))
@ -324,7 +324,7 @@ private:
for (size_t j = 0; j < array_size; ++j)
{
bool hit = false;
if (null_map_data[current_offset + j] == 1)
if (null_map_data[current_offset + j])
{
if (hasNull(null_map_item, i))
hit = true;
@ -394,11 +394,6 @@ struct ArrayIndexNumNullImpl
size_t size = offsets.size();
result.resize(size);
if (!null_map_data)
return;
const auto & null_map_ref = *null_map_data;
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
@ -407,7 +402,7 @@ struct ArrayIndexNumNullImpl
for (size_t j = 0; j < array_size; ++j)
{
if (null_map_ref[current_offset + j] == 1)
if (null_map_data && (*null_map_data)[current_offset + j])
{
if (!IndexConv::apply(j, current))
break;
@ -433,11 +428,6 @@ struct ArrayIndexStringNullImpl
const auto size = offsets.size();
result.resize(size);
if (!null_map_data)
return;
const auto & null_map_ref = *null_map_data;
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
@ -446,8 +436,7 @@ struct ArrayIndexStringNullImpl
for (size_t j = 0; j < array_size; ++j)
{
size_t k = (current_offset == 0 && j == 0) ? 0 : current_offset + j - 1;
if (null_map_ref[k] == 1)
if (null_map_data && (*null_map_data)[current_offset + j])
{
if (!IndexConv::apply(j, current))
break;
@ -487,8 +476,7 @@ struct ArrayIndexStringImpl
ColumnArray::Offset string_size = string_offsets[current_offset + j] - string_pos;
size_t k = (current_offset == 0 && j == 0) ? 0 : current_offset + j - 1;
if (null_map_data && ((*null_map_data)[k] == 1))
if (null_map_data && (*null_map_data)[current_offset + j])
{
}
else if (string_size == value_size + 1 && 0 == memcmp(value.data(), &data[string_pos], value_size))
@ -524,21 +512,20 @@ struct ArrayIndexStringImpl
for (size_t j = 0; j < array_size; ++j)
{
ColumnArray::Offset string_pos = current_offset == 0 && j == 0
? 0
: string_offsets[current_offset + j - 1];
? 0
: string_offsets[current_offset + j - 1];
ColumnArray::Offset string_size = string_offsets[current_offset + j] - string_pos;
bool hit = false;
size_t k = (current_offset == 0 && j == 0) ? 0 : current_offset + j - 1;
if (null_map_data && ((*null_map_data)[k] == 1))
if (null_map_data && (*null_map_data)[current_offset + j])
{
if (null_map_item && ((*null_map_item)[i] == 1))
if (null_map_item && (*null_map_item)[i])
hit = true;
}
else if (string_size == value_size && 0 == memcmp(&item_values[value_pos], &data[string_pos], value_size))
hit = true;
hit = true;
if (hit)
{
@ -638,7 +625,7 @@ private:
for (size_t j = 0; j < array_size; ++j)
{
if (null_map_data[current_offset + j] == 1)
if (null_map_data[current_offset + j])
{
}
else if (0 == data.compareAt(current_offset + j, is_value_has_single_element_to_compare ? 0 : i, value, 1))
@ -674,9 +661,9 @@ private:
for (size_t j = 0; j < array_size; ++j)
{
bool hit = false;
if (null_map_data[current_offset + j] == 1)
if (null_map_data[current_offset + j])
{
if (null_map_item[i] == 1)
if (null_map_item[i])
hit = true;
}
else if (0 == data.compareAt(current_offset + j, is_value_has_single_element_to_compare ? 0 : i, value, 1))
@ -724,11 +711,6 @@ struct ArrayIndexGenericNullImpl
size_t size = offsets.size();
result.resize(size);
if (!null_map_data)
return;
const auto & null_map_ref = *null_map_data;
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
@ -737,7 +719,7 @@ struct ArrayIndexGenericNullImpl
for (size_t j = 0; j < array_size; ++j)
{
if (null_map_ref[current_offset + j] == 1)
if (null_map_data && (*null_map_data)[current_offset + j])
{
if (!IndexConv::apply(j, current))
break;
@ -931,7 +913,7 @@ private:
if (arr[i].isNull())
{
if (null_map && ((*null_map)[row] == 1))
if (null_map && (*null_map)[row])
hit = true;
}
else if (applyVisitor(FieldVisitorAccurateEquals(), arr[i], value))

View File

@ -993,9 +993,17 @@ struct ToIntMonotonicity
{
size_t size_of_type = type.getSizeOfValueInMemory();
/// If type is expanding, then function is monotonic.
/// If type is expanding
if (sizeof(T) > size_of_type)
return { true, true, true };
{
/// If convert signed -> signed or unsigned -> signed, then function is monotonic.
if (std::is_signed_v<T> || type.isValueRepresentedByUnsignedInteger())
return {true, true, true};
/// If arguments from the same half, then function is monotonic.
if ((left.get<Int64>() >= 0) == (right.get<Int64>() >= 0))
return {true, true, true};
}
/// If type is same, too. (Enum has separate case, because it is different data type)
if (checkDataType<DataTypeNumber<T>>(&type) ||

View File

@ -33,6 +33,7 @@
#include <Storages/IStorage.h>
#include <Common/typeid_cast.h>
#include <Storages/getStructureOfRemoteTable.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB
@ -1468,7 +1469,7 @@ private:
/// It is possible to track value from previous block, to calculate continuously across all blocks. Not implemented.
template <typename Src, typename Dst>
static void process(const PaddedPODArray<Src> & src, PaddedPODArray<Dst> & dst)
static void process(const PaddedPODArray<Src> & src, PaddedPODArray<Dst> & dst, const NullMap * null_map)
{
size_t size = src.size();
dst.resize(size);
@ -1478,13 +1479,26 @@ private:
/// It is possible to SIMD optimize this loop. By no need for that in practice.
dst[0] = is_first_line_zero ? 0 : src[0];
Src prev = src[0];
for (size_t i = 1; i < size; ++i)
Src prev;
bool has_prev_value = false;
for (size_t i = 0; i < size; ++i)
{
auto cur = src[i];
dst[i] = static_cast<Dst>(cur) - prev;
prev = cur;
if (null_map && (*null_map)[i])
continue;
if (!has_prev_value)
{
dst[i] = is_first_line_zero ? 0 : src[i];
prev = src[i];
has_prev_value = true;
}
else
{
auto cur = src[i];
dst[i] = static_cast<Dst>(cur) - prev;
prev = cur;
}
}
}
@ -1547,14 +1561,19 @@ public:
return false;
}
bool useDefaultImplementationForNulls() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
DataTypePtr res;
dispatchForSourceType(*arguments[0], [&](auto field_type_tag)
dispatchForSourceType(*removeNullable(arguments[0]), [&](auto field_type_tag)
{
res = std::make_shared<DataTypeNumber<DstFieldType<decltype(field_type_tag)>>>();
});
if (arguments[0]->isNullable())
res = makeNullable(res);
return res;
}
@ -1570,16 +1589,29 @@ public:
return;
}
auto res_column = res_type->createColumn();
auto res_column = removeNullable(res_type)->createColumn();
auto * src_column = src.column.get();
ColumnPtr null_map_column = nullptr;
const NullMap * null_map = nullptr;
if (auto * nullable_column = checkAndGetColumn<ColumnNullable>(src_column))
{
src_column = &nullable_column->getNestedColumn();
null_map_column = nullable_column->getNullMapColumnPtr();
null_map = &nullable_column->getNullMapData();
}
dispatchForSourceType(*src.type, [&](auto field_type_tag)
dispatchForSourceType(*removeNullable(src.type), [&](auto field_type_tag)
{
using SrcFieldType = decltype(field_type_tag);
process(static_cast<const ColumnVector<SrcFieldType> &>(*src.column).getData(),
static_cast<ColumnVector<DstFieldType<SrcFieldType>> &>(*res_column).getData());
process(static_cast<const ColumnVector<SrcFieldType> &>(*src_column).getData(),
static_cast<ColumnVector<DstFieldType<SrcFieldType>> &>(*res_column).getData(), null_map);
});
block.getByPosition(result).column = std::move(res_column);
if (null_map_column)
block.getByPosition(result).column = ColumnNullable::create(std::move(res_column), null_map_column);
else
block.getByPosition(result).column = std::move(res_column);
}
};

View File

@ -40,7 +40,7 @@ namespace
/** Return ColumnNullable of src, with null map as OR-ed null maps of args columns in blocks.
* Or ColumnConst(ColumnNullable) if the result is always NULL or if the result is constant and always not NULL.
*/
ColumnPtr wrapInNullable(const ColumnPtr & src, Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
{
ColumnPtr result_null_map_column;

View File

@ -9,6 +9,7 @@
#include <Common/PODArray.h>
#include <Common/ProfileEvents.h>
#include <Common/Exception.h>
#include <Common/hex.h>
#include <common/unaligned.h>
#include <IO/ReadBuffer.h>
#include <IO/BufferWithOwnMemory.h>
@ -63,7 +64,7 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed,
throw Exception("Unknown compression method: " + toString(method), ErrorCodes::UNKNOWN_COMPRESSION_METHOD);
if (size_compressed > DBMS_MAX_COMPRESSED_SIZE)
throw Exception("Too large size_compressed. Most likely corrupted data.", ErrorCodes::TOO_LARGE_SIZE_COMPRESSED);
throw Exception("Too large size_compressed: " + toString(size_compressed) + ". Most likely corrupted data.", ErrorCodes::TOO_LARGE_SIZE_COMPRESSED);
ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed + sizeof(checksum));
@ -82,8 +83,16 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed,
compressed_in->readStrict(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed - COMPRESSED_BLOCK_HEADER_SIZE);
}
if (!disable_checksum && checksum != CityHash_v1_0_2::CityHash128(compressed_buffer, size_compressed))
throw Exception("Checksum doesn't match: corrupted data.", ErrorCodes::CHECKSUM_DOESNT_MATCH);
if (!disable_checksum)
{
auto checksum_calculated = CityHash_v1_0_2::CityHash128(compressed_buffer, size_compressed);
if (checksum != checksum_calculated)
throw Exception("Checksum doesn't match: corrupted data."
" Reference: " + getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second)
+ ". Actual: " + getHexUIntLowercase(checksum_calculated.first) + getHexUIntLowercase(checksum_calculated.second)
+ ". Size of compressed block: " + toString(size_compressed) + ".",
ErrorCodes::CHECKSUM_DOESNT_MATCH);
}
return size_compressed + sizeof(checksum);
}

View File

@ -3,6 +3,7 @@
#include <Common/DNSResolver.h>
#include <Common/Exception.h>
#include <Common/config.h>
#include <Poco/Version.h>
#if USE_POCO_NETSSL
#include <Poco/Net/AcceptCertificateHandler.h>
#include <Poco/Net/Context.h>
@ -76,7 +77,7 @@ std::unique_ptr<Poco::Net::HTTPClientSession> makeHTTPSession(const Poco::URI &
#if POCO_CLICKHOUSE_PATCH || POCO_VERSION >= 0x02000000
session->setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout);
#else
session->setTimeout(timeouts.connection_timeout);
session->setTimeout(std::max({timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout}));
#endif
return session;

View File

@ -0,0 +1,39 @@
#include <IO/ReadBufferFromIStream.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_READ_FROM_ISTREAM;
}
bool ReadBufferFromIStream::nextImpl()
{
istr.read(internal_buffer.begin(), internal_buffer.size());
size_t gcount = istr.gcount();
if (!gcount)
{
if (istr.eof())
return false;
if (istr.fail())
throw Exception("Cannot read from istream at offset " + std::to_string(count()), ErrorCodes::CANNOT_READ_FROM_ISTREAM);
throw Exception("Unexpected state of istream at offset " + std::to_string(count()), ErrorCodes::CANNOT_READ_FROM_ISTREAM);
}
else
working_buffer.resize(gcount);
return true;
}
ReadBufferFromIStream::ReadBufferFromIStream(std::istream & istr_, size_t size)
: BufferWithOwnMemory<ReadBuffer>(size), istr(istr_)
{
}
}

View File

@ -2,8 +2,6 @@
#include <iostream>
#include <Common/Exception.h>
#include <IO/ReadBuffer.h>
#include <IO/BufferWithOwnMemory.h>
@ -11,38 +9,15 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_READ_FROM_ISTREAM;
}
class ReadBufferFromIStream : public BufferWithOwnMemory<ReadBuffer>
{
private:
std::istream & istr;
bool nextImpl() override
{
istr.read(internal_buffer.begin(), internal_buffer.size());
size_t gcount = istr.gcount();
if (!gcount)
{
if (istr.eof())
return false;
else
throw Exception("Cannot read from istream", ErrorCodes::CANNOT_READ_FROM_ISTREAM);
}
else
working_buffer.resize(gcount);
return true;
}
bool nextImpl() override;
public:
ReadBufferFromIStream(std::istream & istr_, size_t size = DBMS_DEFAULT_BUFFER_SIZE)
: BufferWithOwnMemory<ReadBuffer>(size), istr(istr_) {}
ReadBufferFromIStream(std::istream & istr_, size_t size = DBMS_DEFAULT_BUFFER_SIZE);
};
}

View File

@ -2,6 +2,7 @@
#include <Common/hex.h>
#include <Common/PODArray.h>
#include <Common/StringUtils/StringUtils.h>
#include <Formats/FormatSettings.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/readFloatText.h>
@ -500,18 +501,19 @@ void readBackQuotedStringWithSQLStyle(String & s, ReadBuffer & buf)
template <typename Vector>
void readCSVStringInto(Vector & s, ReadBuffer & buf, const char delimiter)
void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & settings)
{
if (buf.eof())
throwReadAfterEOF();
char maybe_quote = *buf.position();
const char delimiter = settings.delimiter;
const char maybe_quote = *buf.position();
/// Emptiness and not even in quotation marks.
if (maybe_quote == delimiter)
return;
if (maybe_quote == '\'' || maybe_quote == '"')
if ((settings.allow_single_quotes && maybe_quote == '\'') || (settings.allow_double_quotes && maybe_quote == '"'))
{
++buf.position();
@ -575,13 +577,13 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const char delimiter)
}
}
void readCSVString(String & s, ReadBuffer & buf, const char delimiter)
void readCSVString(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings)
{
s.clear();
readCSVStringInto(s, buf, delimiter);
readCSVStringInto(s, buf, settings);
}
template void readCSVStringInto<PaddedPODArray<UInt8>>(PaddedPODArray<UInt8> & s, ReadBuffer & buf, const char delimiter);
template void readCSVStringInto<PaddedPODArray<UInt8>>(PaddedPODArray<UInt8> & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
template <typename Vector, typename ReturnType>

View File

@ -20,6 +20,8 @@
#include <Common/Arena.h>
#include <Common/UInt128.h>
#include <Formats/FormatSettings.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/VarInt.h>
@ -398,7 +400,8 @@ void readStringUntilEOF(String & s, ReadBuffer & buf);
/** Read string in CSV format.
* Parsing rules:
* - string could be placed in quotes; quotes could be single: ' or double: ";
* - string could be placed in quotes; quotes could be single: ' if FormatSettings::CSV::allow_single_quotes is true
* or double: " if FormatSettings::CSV::allow_double_quotes is true;
* - or string could be unquoted - this is determined by first character;
* - if string is unquoted, then it is read until next delimiter,
* either until end of line (CR or LF),
@ -407,7 +410,7 @@ void readStringUntilEOF(String & s, ReadBuffer & buf);
* - if string is in quotes, then it will be read until closing quote,
* but sequences of two consecutive quotes are parsed as single quote inside string;
*/
void readCSVString(String & s, ReadBuffer & buf, const char delimiter);
void readCSVString(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
/// Read and append result to array of characters.
@ -430,7 +433,7 @@ template <typename Vector>
void readStringUntilEOFInto(Vector & s, ReadBuffer & buf);
template <typename Vector>
void readCSVStringInto(Vector & s, ReadBuffer & buf, const char delimiter);
void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
/// ReturnType is either bool or void. If bool, the function will return false instead of throwing an exception.
template <typename Vector, typename ReturnType = void>
@ -688,7 +691,7 @@ template <typename T>
inline std::enable_if_t<std::is_arithmetic_v<T>, void>
readCSV(T & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
inline void readCSV(String & x, ReadBuffer & buf, const char delimiter = ',') { readCSVString(x, buf, delimiter); }
inline void readCSV(String & x, ReadBuffer & buf, const FormatSettings::CSV & settings) { readCSVString(x, buf, settings); }
inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
inline void readCSV(LocalDateTime & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
inline void readCSV(UUID & x, ReadBuffer & buf) { readCSVSimple(x, buf); }

View File

@ -0,0 +1,55 @@
#include <IO/WriteBufferFromOStream.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_WRITE_TO_OSTREAM;
}
void WriteBufferFromOStream::nextImpl()
{
if (!offset())
return;
ostr->write(working_buffer.begin(), offset());
ostr->flush();
if (!ostr->good())
throw Exception("Cannot write to ostream at offset " + std::to_string(count()),
ErrorCodes::CANNOT_WRITE_TO_OSTREAM);
}
WriteBufferFromOStream::WriteBufferFromOStream(
size_t size,
char * existing_memory,
size_t alignment)
: BufferWithOwnMemory<WriteBuffer>(size, existing_memory, alignment)
{
}
WriteBufferFromOStream::WriteBufferFromOStream(
std::ostream & ostr_,
size_t size,
char * existing_memory,
size_t alignment)
: BufferWithOwnMemory<WriteBuffer>(size, existing_memory, alignment), ostr(&ostr_)
{
}
WriteBufferFromOStream::~WriteBufferFromOStream()
{
try
{
next();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}

View File

@ -2,8 +2,6 @@
#include <iostream>
#include <Common/Exception.h>
#include <IO/WriteBuffer.h>
#include <IO/BufferWithOwnMemory.h>
@ -11,53 +9,23 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_WRITE_TO_OSTREAM;
}
class WriteBufferFromOStream : public BufferWithOwnMemory<WriteBuffer>
{
protected:
std::ostream * ostr;
void nextImpl() override
{
if (!offset())
return;
ostr->write(working_buffer.begin(), offset());
ostr->flush();
if (!ostr->good())
throw Exception("Cannot write to ostream", ErrorCodes::CANNOT_WRITE_TO_OSTREAM);
}
WriteBufferFromOStream(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0)
: BufferWithOwnMemory<WriteBuffer>(size, existing_memory, alignment)
{
}
void nextImpl() override;
WriteBufferFromOStream(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0);
public:
WriteBufferFromOStream(
std::ostream & ostr_,
size_t size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0)
: BufferWithOwnMemory<WriteBuffer>(size, existing_memory, alignment), ostr(&ostr_) {}
size_t alignment = 0);
~WriteBufferFromOStream() override
{
try
{
next();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
~WriteBufferFromOStream() override;
};
}

View File

@ -1731,9 +1731,9 @@ void Context::setFormatSchemaPath(const String & path)
shared->format_schema_path = path;
}
Context::getSampleBlockCacheType & Context::getSampleBlockCache() const
Context::SampleBlockCache & Context::getSampleBlockCache() const
{
return getQueryContext().get_sample_block_cache;
return getQueryContext().sample_block_cache;
}
std::shared_ptr<ActionLocksManager> Context::getActionLocksManager()

View File

@ -121,6 +121,9 @@ private:
UInt64 session_close_cycle = 0;
bool session_is_used = false;
using SampleBlockCache = std::unordered_map<std::string, Block>;
mutable SampleBlockCache sample_block_cache;
using DatabasePtr = std::shared_ptr<IDatabase>;
using Databases = std::map<String, std::shared_ptr<IDatabase>>;
@ -401,9 +404,7 @@ public:
/// User name and session identifier. Named sessions are local to users.
using SessionKey = std::pair<String, String>;
using getSampleBlockCacheType = std::unordered_map<std::string, Block>;
mutable Context::getSampleBlockCacheType get_sample_block_cache;
getSampleBlockCacheType & getSampleBlockCache() const;
SampleBlockCache & getSampleBlockCache() const;
private:
/** Check if the current client has access to the specified database.

View File

@ -1473,17 +1473,29 @@ void ExpressionAnalyzer::makeSetsForIndex()
}
void ExpressionAnalyzer::tryMakeSetFromSubquery(const ASTPtr & subquery_or_table_name)
void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name)
{
BlockIO res = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {})->execute();
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
SizeLimits set_for_index_size_limits;
if (settings.use_index_for_in_with_subqueries_max_values && settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set)
{
/// Silently cancel creating the set for index if the specific limit has been reached.
set_for_index_size_limits = SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK);
}
else
{
/// If the limit specific for set for index is lower than general limits for set - use general limit.
set_for_index_size_limits = SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode);
}
SetPtr set = std::make_shared<Set>(set_for_index_size_limits, true);
set->setHeader(res.in->getHeader());
while (Block block = res.in->read())
{
/// If the limits have been exceeded, give up and let the default subquery processing actions take place.
if (!set->insertFromBlock(block, true))
if (!set->insertFromBlock(block))
return;
}
@ -1521,7 +1533,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
if (typeid_cast<ASTSubquery *>(arg.get()) || typeid_cast<ASTIdentifier *>(arg.get()))
{
if (settings.use_index_for_in_with_subqueries)
tryMakeSetFromSubquery(arg);
tryMakeSetForIndexFromSubquery(arg);
}
else
{
@ -1589,7 +1601,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
return;
}
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), false);
/** The following happens for GLOBAL INs:
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
@ -1711,8 +1723,8 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block &
+ left_arg_type->getName() + " and " + right_arg_type->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
set->createFromAST(set_element_types, elements_ast, context, create_ordered_set);
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), create_ordered_set);
set->createFromAST(set_element_types, elements_ast, context);
prepared_sets[right_arg->range] = std::move(set);
}

View File

@ -345,7 +345,7 @@ private:
* Create Set from a subuqery or a table expression in the query. The created set is suitable for using the index.
* The set will not be created if its size hits the limit.
*/
void tryMakeSetFromSubquery(const ASTPtr & subquery_or_table_name);
void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name);
void makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block);

View File

@ -18,9 +18,6 @@ BlockIO InterpreterOptimizeQuery::execute()
{
const ASTOptimizeQuery & ast = typeid_cast<const ASTOptimizeQuery &>(*query_ptr);
if (ast.final && !ast.partition)
throw Exception("FINAL flag for OPTIMIZE query is meaningful only with specified PARTITION", ErrorCodes::BAD_ARGUMENTS);
StoragePtr table = context.getTable(ast.database, ast.table);
auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__);
table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context);

View File

@ -1,3 +1,5 @@
#include <optional>
#include <Core/Field.h>
#include <Common/FieldVisitors.h>
#include <Core/Row.h>
@ -20,9 +22,12 @@
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/NullableUtils.h>
#include <Interpreters/sortBlock.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <ext/range.h>
namespace DB
{
@ -43,22 +48,34 @@ void NO_INLINE Set::insertFromBlockImpl(
const ColumnRawPtrs & key_columns,
size_t rows,
SetVariants & variants,
ConstNullMapPtr null_map)
ConstNullMapPtr null_map,
ColumnUInt8::Container * out_filter)
{
if (null_map)
insertFromBlockImplCase<Method, true>(method, key_columns, rows, variants, null_map);
{
if (out_filter)
insertFromBlockImplCase<Method, true, true>(method, key_columns, rows, variants, null_map, out_filter);
else
insertFromBlockImplCase<Method, true, false>(method, key_columns, rows, variants, null_map, out_filter);
}
else
insertFromBlockImplCase<Method, false>(method, key_columns, rows, variants, null_map);
{
if (out_filter)
insertFromBlockImplCase<Method, false, true>(method, key_columns, rows, variants, null_map, out_filter);
else
insertFromBlockImplCase<Method, false, false>(method, key_columns, rows, variants, null_map, out_filter);
}
}
template <typename Method, bool has_null_map>
template <typename Method, bool has_null_map, bool build_filter>
void NO_INLINE Set::insertFromBlockImplCase(
Method & method,
const ColumnRawPtrs & key_columns,
size_t rows,
SetVariants & variants,
ConstNullMapPtr null_map)
ConstNullMapPtr null_map,
ColumnUInt8::Container * out_filter)
{
typename Method::State state;
state.init(key_columns);
@ -78,6 +95,9 @@ void NO_INLINE Set::insertFromBlockImplCase(
if (inserted)
method.onNewKey(*it, keys_size, variants.string_pool);
if (build_filter)
(*out_filter)[i] = inserted;
}
}
@ -115,12 +135,22 @@ void Set::setHeader(const Block & block)
ConstNullMapPtr null_map{};
extractNestedColumnsAndNullMap(key_columns, null_map_holder, null_map);
if (fill_set_elements)
{
/// Create empty columns with set values in advance.
/// It is needed because set may be empty, so method 'insertFromBlock' will be never called.
set_elements.reserve(keys_size);
for (const auto & type : data_types)
set_elements.emplace_back(removeNullable(type)->createColumn());
}
/// Choose data structure to use for the set.
data.init(data.chooseMethod(key_columns, key_sizes));
}
bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
bool Set::insertFromBlock(const Block & block)
{
std::unique_lock lock(rwlock);
@ -152,13 +182,18 @@ bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
ConstNullMapPtr null_map{};
extractNestedColumnsAndNullMap(key_columns, null_map_holder, null_map);
/// Filter to extract distinct values from the block.
ColumnUInt8::MutablePtr filter;
if (fill_set_elements)
filter = ColumnUInt8::create(block.rows());
switch (data.type)
{
case SetVariants::Type::EMPTY:
break;
#define M(NAME) \
case SetVariants::Type::NAME: \
insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map); \
insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, filter ? &filter->getData() : nullptr); \
break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
@ -166,13 +201,13 @@ bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
if (fill_set_elements)
{
for (size_t i = 0; i < rows; ++i)
for (size_t i = 0; i < keys_size; ++i)
{
std::vector<Field> new_set_elements;
for (size_t j = 0; j < keys_size; ++j)
new_set_elements.push_back((*key_columns[j])[i]);
set_elements->emplace_back(std::move(new_set_elements));
auto filtered_column = block.getByPosition(i).column->filter(filter->getData(), rows);
if (set_elements[i]->empty())
set_elements[i] = filtered_column;
else
set_elements[i]->assumeMutableRef().insertRangeFrom(*filtered_column, 0, filtered_column->size());
}
}
@ -196,7 +231,7 @@ static Field extractValueFromNode(ASTPtr & node, const IDataType & type, const C
}
void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & context, bool fill_set_elements)
void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & context)
{
/// Will form a block with values from the set.
@ -267,7 +302,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
}
Block block = header.cloneWithColumns(std::move(columns));
insertFromBlock(block, fill_set_elements);
insertFromBlock(block);
}
@ -403,9 +438,8 @@ void Set::executeOrdinary(
}
MergeTreeSetIndex::MergeTreeSetIndex(const SetElements & set_elements, std::vector<KeyTuplePositionMapping> && index_mapping_)
: ordered_set(),
indexes_mapping(std::move(index_mapping_))
MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector<KeyTuplePositionMapping> && index_mapping_)
: indexes_mapping(std::move(index_mapping_))
{
std::sort(indexes_mapping.begin(), indexes_mapping.end(),
[](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r)
@ -420,16 +454,23 @@ MergeTreeSetIndex::MergeTreeSetIndex(const SetElements & set_elements, std::vect
return l.key_index == r.key_index;
}), indexes_mapping.end());
for (size_t i = 0; i < set_elements.size(); ++i)
{
std::vector<FieldWithInfinity> new_set_values;
for (size_t j = 0; j < indexes_mapping.size(); ++j)
new_set_values.emplace_back(set_elements[i][indexes_mapping[j].tuple_index]);
size_t tuple_size = indexes_mapping.size();
ordered_set.resize(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)
ordered_set[i] = set_elements[indexes_mapping[i].tuple_index];
ordered_set.emplace_back(std::move(new_set_values));
Block block_to_sort;
SortDescription sort_description;
for (size_t i = 0; i < tuple_size; ++i)
{
block_to_sort.insert({ ordered_set[i], nullptr, "" });
sort_description.emplace_back(i, 1, 1);
}
std::sort(ordered_set.begin(), ordered_set.end());
sortBlock(block_to_sort, sort_description);
for (size_t i = 0; i < tuple_size; ++i)
ordered_set[i] = block_to_sort.getByPosition(i).column;
}
@ -439,15 +480,19 @@ MergeTreeSetIndex::MergeTreeSetIndex(const SetElements & set_elements, std::vect
*/
BoolMask MergeTreeSetIndex::mayBeTrueInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types)
{
std::vector<FieldWithInfinity> left_point;
std::vector<FieldWithInfinity> right_point;
left_point.reserve(indexes_mapping.size());
right_point.reserve(indexes_mapping.size());
size_t tuple_size = indexes_mapping.size();
using FieldWithInfinityTuple = std::vector<FieldWithInfinity>;
FieldWithInfinityTuple left_point;
FieldWithInfinityTuple right_point;
left_point.reserve(tuple_size);
right_point.reserve(tuple_size);
bool invert_left_infinities = false;
bool invert_right_infinities = false;
for (size_t i = 0; i < indexes_mapping.size(); ++i)
for (size_t i = 0; i < tuple_size; ++i)
{
std::optional<Range> new_range = KeyCondition::applyMonotonicFunctionsChainToRange(
key_ranges[indexes_mapping[i].key_index],
@ -491,16 +536,40 @@ BoolMask MergeTreeSetIndex::mayBeTrueInRange(const std::vector<Range> & key_rang
}
}
/// This allows to construct tuple in 'ordered_set' at specified index for comparison with range.
auto indices = ext::range(0, ordered_set.at(0)->size());
auto extract_tuple = [tuple_size, this](size_t i)
{
/// Inefficient.
FieldWithInfinityTuple res;
res.reserve(tuple_size);
for (size_t j = 0; j < tuple_size; ++j)
res.emplace_back((*ordered_set[j])[i]);
return res;
};
auto compare = [&extract_tuple](size_t i, const FieldWithInfinityTuple & rhs)
{
return extract_tuple(i) < rhs;
};
/** Because each parallelogram maps to a contiguous sequence of elements
* layed out in the lexicographically increasing order, the set intersects the range
* if and only if either bound coincides with an element or at least one element
* is between the lower bounds
*/
auto left_lower = std::lower_bound(ordered_set.begin(), ordered_set.end(), left_point);
auto right_lower = std::lower_bound(ordered_set.begin(), ordered_set.end(), right_point);
return {left_lower != right_lower
|| (left_lower != ordered_set.end() && *left_lower == left_point)
|| (right_lower != ordered_set.end() && *right_lower == right_point), true};
auto left_lower = std::lower_bound(indices.begin(), indices.end(), left_point, compare);
auto right_lower = std::lower_bound(indices.begin(), indices.end(), right_point, compare);
return
{
left_lower != right_lower
|| (left_lower != indices.end() && extract_tuple(*left_lower) == left_point)
|| (right_lower != indices.end() && extract_tuple(*right_lower) == right_point),
true
};
}
}

View File

@ -18,21 +18,22 @@ namespace DB
struct Range;
class FieldWithInfinity;
using SetElements = std::vector<std::vector<Field>>;
using SetElementsPtr = std::unique_ptr<SetElements>;
class IFunctionBase;
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
/** Data structure for implementation of IN expression.
*/
class Set
{
public:
Set(const SizeLimits & limits) :
log(&Logger::get("Set")),
limits(limits),
set_elements(std::make_unique<SetElements>())
/// 'fill_set_elements': in addition to hash table
/// (that is useful only for checking that some value is in the set and may not store the original values),
/// store all set elements in explicit form.
/// This is needed for subsequent use for index.
Set(const SizeLimits & limits, bool fill_set_elements)
: log(&Logger::get("Set")),
limits(limits), fill_set_elements(fill_set_elements)
{
}
@ -46,7 +47,7 @@ public:
* 'node' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
* 'fill_set_elements' - if true, fill vector of elements. For primary key to work.
*/
void createFromAST(const DataTypes & types, ASTPtr node, const Context & context, bool fill_set_elements);
void createFromAST(const DataTypes & types, ASTPtr node, const Context & context);
/** Create a Set from stream.
* Call setHeader, then call insertFromBlock for each block.
@ -54,7 +55,7 @@ public:
void setHeader(const Block & header);
/// Returns false, if some limit was exceeded and no need to insert more data.
bool insertFromBlock(const Block & block, bool fill_set_elements);
bool insertFromBlock(const Block & block);
/** For columns of 'block', check belonging of corresponding rows to the set.
* Return UInt8 column with the result.
@ -66,7 +67,8 @@ public:
const DataTypes & getDataTypes() const { return data_types; }
SetElements & getSetElements() { return *set_elements.get(); }
bool hasExplicitSetElements() const { return fill_set_elements; }
const Columns & getSetElements() const { return set_elements; }
private:
size_t keys_size = 0;
@ -99,6 +101,9 @@ private:
/// Limitations on the maximum size of the set
SizeLimits limits;
/// Do we need to additionally store all elements of the set in explicit form for subsequent use for index.
bool fill_set_elements;
/// If in the left part columns contains the same types as the elements of the set.
void executeOrdinary(
const ColumnRawPtrs & key_columns,
@ -106,9 +111,9 @@ private:
bool negative,
const PaddedPODArray<UInt8> * null_map) const;
/// Vector of elements of `Set`.
/// Collected elements of `Set`.
/// It is necessary for the index to work on the primary key in the IN statement.
SetElementsPtr set_elements;
Columns set_elements;
/** Protects work with the set in the functions `insertFromBlock` and `execute`.
* These functions can be called simultaneously from different threads only when using StorageSet,
@ -123,15 +128,17 @@ private:
const ColumnRawPtrs & key_columns,
size_t rows,
SetVariants & variants,
ConstNullMapPtr null_map);
ConstNullMapPtr null_map,
ColumnUInt8::Container * out_filter);
template <typename Method, bool has_null_map>
template <typename Method, bool has_null_map, bool build_filter>
void insertFromBlockImplCase(
Method & method,
const ColumnRawPtrs & key_columns,
size_t rows,
SetVariants & variants,
ConstNullMapPtr null_map);
ConstNullMapPtr null_map,
ColumnUInt8::Container * out_filter);
template <typename Method>
void executeImpl(
@ -156,6 +163,7 @@ using SetPtr = std::shared_ptr<Set>;
using ConstSetPtr = std::shared_ptr<const Set>;
using Sets = std::vector<SetPtr>;
class IFunction;
using FunctionPtr = std::shared_ptr<IFunction>;
@ -164,8 +172,7 @@ class MergeTreeSetIndex
{
public:
/** Mapping for tuple positions from Set::set_elements to
* position of pk index and data type of this pk column
* and functions chain applied to this column.
* position of pk index and functions chain applied to this column.
*/
struct KeyTuplePositionMapping
{
@ -174,16 +181,14 @@ public:
std::vector<FunctionBasePtr> functions;
};
MergeTreeSetIndex(const SetElements & set_elements, std::vector<KeyTuplePositionMapping> && indexes_mapping_);
MergeTreeSetIndex(const Columns & set_elements, std::vector<KeyTuplePositionMapping> && indexes_mapping_);
size_t size() const { return ordered_set.size(); }
size_t size() const { return ordered_set.at(0)->size(); }
BoolMask mayBeTrueInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types);
private:
using OrderedTuples = std::vector<std::vector<FieldWithInfinity>>;
OrderedTuples ordered_set;
Columns ordered_set;
std::vector<KeyTuplePositionMapping> indexes_mapping;
};

View File

@ -33,32 +33,32 @@ struct Settings
*/
#define APPLY_FOR_SETTINGS(M) \
M(SettingUInt64, min_compress_block_size, DEFAULT_MIN_COMPRESS_BLOCK_SIZE, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.") \
M(SettingUInt64, max_compress_block_size, DEFAULT_MAX_COMPRESS_BLOCK_SIZE, "The maximum size of blocks of uncompressed data before compressing for writing to a table.") \
M(SettingUInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.") \
M(SettingUInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.") \
M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading") \
M(SettingUInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.") \
M(SettingUInt64, min_insert_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.") \
M(SettingUInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.") \
M(SettingMaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.") \
M(SettingUInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.") \
M(SettingUInt64, max_distributed_connections, DEFAULT_MAX_DISTRIBUTED_CONNECTIONS, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).") \
M(SettingUInt64, max_query_size, DEFAULT_MAX_QUERY_SIZE, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)") \
M(SettingUInt64, interactive_delay, DEFAULT_INTERACTIVE_DELAY, "The interval in microseconds to check if the request is cancelled, and to send progress info.") \
M(SettingUInt64, max_distributed_connections, 1024, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).") \
M(SettingUInt64, max_query_size, 262144, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)") \
M(SettingUInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.") \
M(SettingSeconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.") \
M(SettingMilliseconds, connect_timeout_with_failover_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS, "Connection timeout for selecting first healthy replica.") \
M(SettingSeconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "") \
M(SettingSeconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "") \
M(SettingMilliseconds, queue_max_wait_ms, DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.") \
M(SettingMilliseconds, queue_max_wait_ms, 5000, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.") \
M(SettingUInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.") \
M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.") \
M(SettingUInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.") \
M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.") \
M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.") \
M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.") \
M(SettingUInt64, background_pool_size, DBMS_DEFAULT_BACKGROUND_POOL_SIZE, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.") \
M(SettingUInt64, background_schedule_pool_size, DBMS_DEFAULT_BACKGROUND_POOL_SIZE, "Number of threads performing background tasks for replicated tables. Only has meaning at server startup.") \
M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.") \
M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables. Only has meaning at server startup.") \
\
M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS, "Sleep time for StorageDistributed DirectoryMonitors in case there is no work or exception has been thrown.") \
M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors in case there is no work or exception has been thrown.") \
\
M(SettingBool, distributed_directory_monitor_batch_inserts, false, "Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones.") \
\
@ -188,6 +188,7 @@ struct Settings
M(SettingSeconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout") \
M(SettingBool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown") \
M(SettingBool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.") \
M(SettingUInt64, use_index_for_in_with_subqueries_max_values, 100000, "Don't use index of a table for filtering by right hand size of the IN operator if the size of set is larger than specified threshold. This allows to avoid performance degradation and higher memory usage due to preparation of additional data structures.") \
\
M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.") \
M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.") \
@ -262,6 +263,9 @@ struct Settings
M(SettingUInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.")\
M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.") \
M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.") \
M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.") \
M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.") \
\
M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \
\
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \

View File

@ -121,7 +121,7 @@ void sortBlock(Block & block, const SortDescription & description, size_t limit)
size_t columns = block.columns();
for (size_t i = 0; i < columns; ++i)
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->permute(perm, limit);
block.getByPosition(i).column = block.getByPosition(i).column->permute(perm, limit);
}
else
{
@ -166,7 +166,7 @@ void sortBlock(Block & block, const SortDescription & description, size_t limit)
size_t columns = block.columns();
for (size_t i = 0; i < columns; ++i)
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->permute(perm, limit);
block.getByPosition(i).column = block.getByPosition(i).column->permute(perm, limit);
}
}

View File

@ -1,4 +1,4 @@
#include <Storages/MergeTree/AbandonableLockInZooKeeper.h>
#include <Storages/MergeTree/EphemeralLockInZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <common/logger_useful.h>
@ -11,59 +11,59 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
AbandonableLockInZooKeeper::AbandonableLockInZooKeeper(
EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(
const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, zkutil::Requests * precheck_ops)
: zookeeper(&zookeeper_), path_prefix(path_prefix_)
{
String abandonable_path = temp_path + "/abandonable_lock-";
/// The /abandonable_lock- name is for backward compatibility.
String holder_path_prefix = temp_path + "/abandonable_lock-";
/// Let's create an secondary ephemeral node.
if (!precheck_ops || precheck_ops->empty())
{
holder_path = zookeeper->create(abandonable_path, "", zkutil::CreateMode::EphemeralSequential);
holder_path = zookeeper->create(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential);
}
else
{
precheck_ops->emplace_back(zkutil::makeCreateRequest(abandonable_path, "", zkutil::CreateMode::EphemeralSequential));
precheck_ops->emplace_back(zkutil::makeCreateRequest(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential));
zkutil::Responses op_results = zookeeper->multi(*precheck_ops);
holder_path = dynamic_cast<const zkutil::CreateResponse &>(*op_results.back()).path_created;
}
/// Write the path to the secondary node in the main node.
path = zookeeper->create(path_prefix, holder_path, zkutil::CreateMode::PersistentSequential);
path = zookeeper->create(path_prefix, holder_path, zkutil::CreateMode::EphemeralSequential);
if (path.size() <= path_prefix.size())
throw Exception("Logical error: name of sequential node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Logical error: name of the main node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR);
}
void AbandonableLockInZooKeeper::unlock()
void EphemeralLockInZooKeeper::unlock()
{
checkCreated();
zookeeper->remove(path);
zookeeper->remove(holder_path);
zkutil::Requests ops;
getUnlockOps(ops);
zookeeper->multi(ops);
holder_path = "";
}
void AbandonableLockInZooKeeper::getUnlockOps(zkutil::Requests & ops)
void EphemeralLockInZooKeeper::getUnlockOps(zkutil::Requests & ops)
{
checkCreated();
ops.emplace_back(zkutil::makeRemoveRequest(path, -1));
ops.emplace_back(zkutil::makeRemoveRequest(holder_path, -1));
}
AbandonableLockInZooKeeper::~AbandonableLockInZooKeeper()
EphemeralLockInZooKeeper::~EphemeralLockInZooKeeper()
{
if (!zookeeper || holder_path.empty())
if (!isCreated())
return;
try
{
zookeeper->tryRemove(holder_path);
zookeeper->trySet(path, ""); /// It's not strictly necessary.
unlock();
}
catch (...)
{
tryLogCurrentException("~AbandonableLockInZooKeeper");
tryLogCurrentException("~EphemeralLockInZooKeeper");
}
}

View File

@ -13,33 +13,24 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
/** The synchronization is primitive. Works as follows:
* Creates a non-ephemeral incremental node and marks it as locked (LOCKED).
* `unlock()` unlocks it (UNLOCKED).
* When the destructor is called or the session ends in ZooKeeper, it goes into the ABANDONED state.
* (Including when the program is halted).
*/
class AbandonableLockInZooKeeper : public boost::noncopyable
/// A class that is used for locking a block number in a partition.
/// It creates a secondary ephemeral node in `temp_path` and a main ephemeral node with `path_prefix`
/// that references the secondary node. The reasons for this two-level scheme are historical (of course
/// it would be simpler to allocate block numbers for all partitions in one ZK directory).
class EphemeralLockInZooKeeper : public boost::noncopyable
{
public:
enum State
{
UNLOCKED,
LOCKED,
ABANDONED,
};
AbandonableLockInZooKeeper(
EphemeralLockInZooKeeper(
const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, zkutil::Requests * precheck_ops = nullptr);
AbandonableLockInZooKeeper() = default;
EphemeralLockInZooKeeper() = default;
AbandonableLockInZooKeeper(AbandonableLockInZooKeeper && rhs) noexcept
EphemeralLockInZooKeeper(EphemeralLockInZooKeeper && rhs) noexcept
{
*this = std::move(rhs);
}
AbandonableLockInZooKeeper & operator=(AbandonableLockInZooKeeper && rhs) noexcept
EphemeralLockInZooKeeper & operator=(EphemeralLockInZooKeeper && rhs) noexcept
{
zookeeper = rhs.zookeeper;
rhs.zookeeper = nullptr;
@ -82,10 +73,10 @@ public:
void checkCreated() const
{
if (!isCreated())
throw Exception("AbandonableLock is not created", ErrorCodes::LOGICAL_ERROR);
throw Exception("EphemeralLock is not created", ErrorCodes::LOGICAL_ERROR);
}
~AbandonableLockInZooKeeper();
~EphemeralLockInZooKeeper();
private:
zkutil::ZooKeeper * zookeeper = nullptr;
@ -95,8 +86,7 @@ private:
};
/// Acquires block number locks in all partitions. The class is called Ephemeral- instead of Abandonable-
/// because it creates ephemeral block nodes (there is no need to leave abandoned tombstones).
/// Acquires block number locks in all partitions.
class EphemeralLocksInAllPartitions
{
public:

View File

@ -269,13 +269,13 @@ KeyCondition::KeyCondition(
const SelectQueryInfo & query_info,
const Context & context,
const NamesAndTypesList & all_columns,
const SortDescription & sort_descr_,
const Names & key_column_names,
const ExpressionActionsPtr & key_expr_)
: sort_descr(sort_descr_), key_expr(key_expr_), prepared_sets(query_info.sets)
: key_expr(key_expr_), prepared_sets(query_info.sets)
{
for (size_t i = 0; i < sort_descr.size(); ++i)
for (size_t i = 0, size = key_column_names.size(); i < size; ++i)
{
std::string name = sort_descr[i].column_name;
std::string name = key_column_names[i];
if (!key_columns.count(name))
key_columns[name] = i;
}
@ -484,14 +484,17 @@ void KeyCondition::getKeyTuplePositionMapping(
}
/// Try to prepare KeyTuplePositionMapping for tuples from IN expression.
bool KeyCondition::isTupleIndexable(
bool KeyCondition::tryPrepareSetIndex(
const ASTPtr & node,
const Context & context,
RPNElement & out,
const SetPtr & prepared_set,
size_t & out_key_column_num)
{
/// The index can be prepared if the elements of the set were saved in advance.
if (!prepared_set->hasExplicitSetElements())
return false;
out_key_column_num = 0;
std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> indexes_mapping;
@ -523,8 +526,7 @@ bool KeyCondition::isTupleIndexable(
if (indexes_mapping.empty())
return false;
out.set_index = std::make_shared<MergeTreeSetIndex>(
prepared_set->getSetElements(), std::move(indexes_mapping));
out.set_index = std::make_shared<MergeTreeSetIndex>(prepared_set->getSetElements(), std::move(indexes_mapping));
return true;
}
@ -636,13 +638,13 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
DataTypePtr key_expr_type; /// Type of expression containing key column
size_t key_arg_pos; /// Position of argument with key column (non-const argument)
size_t key_column_num; /// Number of a key column (inside sort_descr array)
size_t key_column_num; /// Number of a key column (inside key_column_names array)
MonotonicFunctionsChain chain;
bool is_set_const = false;
bool is_constant_transformed = false;
if (prepared_sets.count(args[1]->range)
&& isTupleIndexable(args[0], context, out, prepared_sets[args[1]->range], key_column_num))
&& tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->range], key_column_num))
{
key_arg_pos = 0;
is_set_const = true;

View File

@ -189,6 +189,7 @@ public:
String toString() const;
};
/// Class that extends arbitrary objects with infinities, like +-inf for floats
class FieldWithInfinity
{
@ -216,6 +217,7 @@ private:
FieldWithInfinity(const Type type_);
};
/** Condition on the index.
*
* Consists of the conditions for the key belonging to all possible ranges or sets,
@ -232,7 +234,7 @@ public:
const SelectQueryInfo & query_info,
const Context & context,
const NamesAndTypesList & all_columns,
const SortDescription & sort_descr,
const Names & key_column_names,
const ExpressionActionsPtr & key_expr);
/// Whether the condition is feasible in the key range.
@ -324,8 +326,8 @@ private:
public:
static const AtomMap atom_map;
private:
private:
bool mayBeTrueInRange(
size_t used_key_size,
const Field * left_key,
@ -370,7 +372,10 @@ private:
const size_t tuple_index,
size_t & out_key_column_num);
bool isTupleIndexable(
/// If it's possible to make an RPNElement
/// that will filter values (possibly tuples) by the content of 'prepared_set',
/// do it and return true.
bool tryPrepareSetIndex(
const ASTPtr & node,
const Context & context,
RPNElement & out,
@ -379,7 +384,6 @@ private:
RPN rpn;
SortDescription sort_descr;
ColumnIndices key_columns;
ExpressionActionsPtr key_expr;
PreparedSets prepared_sets;

View File

@ -159,8 +159,9 @@ MergeTreeData::MergeTreeData(
Poco::File(full_path + "detached").createDirectory();
String version_file_path = full_path + "format_version.txt";
// When data path not exists, ignore the format_version check
if (!attach || !path_exists)
auto version_file_exists = Poco::File(version_file_path).exists();
// When data path or file not exists, ignore the format_version check
if (!attach || !path_exists || !version_file_exists)
{
format_version = min_format_version;
WriteBufferFromFile buf(version_file_path);
@ -214,19 +215,16 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam
void MergeTreeData::initPrimaryKey()
{
auto addSortDescription = [](SortDescription & descr, const ASTPtr & expr_ast)
auto addSortColumns = [](Names & out, const ASTPtr & expr_ast)
{
descr.reserve(descr.size() + expr_ast->children.size());
out.reserve(out.size() + expr_ast->children.size());
for (const ASTPtr & ast : expr_ast->children)
{
String name = ast->getColumnName();
descr.emplace_back(name, 1, 1);
}
out.emplace_back(ast->getColumnName());
};
/// Initialize description of sorting for primary key.
primary_sort_descr.clear();
addSortDescription(primary_sort_descr, primary_expr_ast);
primary_sort_columns.clear();
addSortColumns(primary_sort_columns, primary_expr_ast);
primary_expr = ExpressionAnalyzer(primary_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false);
@ -243,10 +241,10 @@ void MergeTreeData::initPrimaryKey()
for (size_t i = 0; i < primary_key_size; ++i)
primary_key_data_types[i] = primary_key_sample.getByPosition(i).type;
sort_descr = primary_sort_descr;
sort_columns = primary_sort_columns;
if (secondary_sort_expr_ast)
{
addSortDescription(sort_descr, secondary_sort_expr_ast);
addSortColumns(sort_columns, secondary_sort_expr_ast);
secondary_sort_expr = ExpressionAnalyzer(secondary_sort_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false);
ExpressionActionsPtr projected_expr =
@ -279,7 +277,6 @@ void MergeTreeData::initPartitionKey()
{
minmax_idx_columns.emplace_back(column.name);
minmax_idx_column_types.emplace_back(column.type);
minmax_idx_sort_descr.emplace_back(column.name, 1, 1);
}
/// Try to find the date column in columns used by the partition key (a common case).
@ -2282,14 +2279,14 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData:
bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const
{
String column_name = node->getColumnName();
const String column_name = node->getColumnName();
for (const auto & column : primary_sort_descr)
if (column_name == column.column_name)
for (const auto & name : primary_sort_columns)
if (column_name == name)
return true;
for (const auto & column : minmax_idx_sort_descr)
if (column_name == column.column_name)
for (const auto & name : minmax_idx_columns)
if (column_name == name)
return true;
if (const ASTFunction * func = typeid_cast<const ASTFunction *>(node.get()))

View File

@ -1,6 +1,5 @@
#pragma once
#include <Core/SortDescription.h>
#include <Common/SimpleIncrement.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
@ -480,11 +479,11 @@ public:
broken_part_callback(name);
}
bool hasPrimaryKey() const { return !primary_sort_descr.empty(); }
bool hasPrimaryKey() const { return !primary_sort_columns.empty(); }
ExpressionActionsPtr getPrimaryExpression() const { return primary_expr; }
ExpressionActionsPtr getSecondarySortExpression() const { return secondary_sort_expr; } /// may return nullptr
SortDescription getPrimarySortDescription() const { return primary_sort_descr; }
SortDescription getSortDescription() const { return sort_descr; }
Names getPrimarySortColumns() const { return primary_sort_columns; }
Names getSortColumns() const { return sort_columns; }
/// Check that the part is not broken and calculate the checksums for it if they are not present.
MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path);
@ -555,7 +554,6 @@ public:
Names minmax_idx_columns;
DataTypes minmax_idx_column_types;
Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column.
SortDescription minmax_idx_sort_descr; /// For use with KeyCondition.
/// Limiting parallel sends per one table, used in DataPartsExchange
std::atomic_uint current_table_sends {0};
@ -576,10 +574,10 @@ private:
ExpressionActionsPtr primary_expr;
/// Additional expression for sorting (of rows with the same primary keys).
ExpressionActionsPtr secondary_sort_expr;
/// Sort description for primary key. Is the prefix of sort_descr.
SortDescription primary_sort_descr;
/// Sort description for primary key + secondary sorting columns.
SortDescription sort_descr;
/// Names of columns for primary key. Is the prefix of sort_columns.
Names primary_sort_columns;
/// Names of columns for primary key + secondary sorting columns.
Names sort_columns;
String database_name;
String table_name;

View File

@ -243,7 +243,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge(
bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition(
FuturePart & future_part,
size_t available_disk_space,
size_t & available_disk_space,
const AllowedMergingPredicate & can_merge,
const String & partition_id,
bool final,
@ -306,6 +306,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition(
LOG_DEBUG(log, "Selected " << parts.size() << " parts from " << parts.front()->name << " to " << parts.back()->name);
future_part.assign(std::move(parts));
available_disk_space -= required_disk_space;
return true;
}
@ -549,7 +550,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
Names all_column_names = data.getColumns().getNamesOfPhysical();
NamesAndTypesList all_columns = data.getColumns().getAllPhysical();
const SortDescription sort_desc = data.getSortDescription();
NamesAndTypesList gathering_columns, merging_columns;
Names gathering_column_names, merging_column_names;
@ -611,6 +611,15 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
src_streams.emplace_back(std::move(input));
}
Names sort_columns = data.getSortColumns();
SortDescription sort_description;
size_t sort_columns_size = sort_columns.size();
sort_description.reserve(sort_columns_size);
Block header = src_streams.at(0)->getHeader();
for (size_t i = 0; i < sort_columns_size; ++i)
sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1);
/// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number.
/// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part,
/// that is going in insertion order.
@ -620,38 +629,38 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
{
case MergeTreeData::MergingParams::Ordinary:
merged_stream = std::make_unique<MergingSortedBlockInputStream>(
src_streams, sort_desc, DEFAULT_MERGE_BLOCK_SIZE, 0, rows_sources_write_buf.get(), true);
src_streams, sort_description, DEFAULT_MERGE_BLOCK_SIZE, 0, rows_sources_write_buf.get(), true);
break;
case MergeTreeData::MergingParams::Collapsing:
merged_stream = std::make_unique<CollapsingSortedBlockInputStream>(
src_streams, sort_desc, data.merging_params.sign_column, DEFAULT_MERGE_BLOCK_SIZE, rows_sources_write_buf.get());
src_streams, sort_description, data.merging_params.sign_column, DEFAULT_MERGE_BLOCK_SIZE, rows_sources_write_buf.get());
break;
case MergeTreeData::MergingParams::Summing:
merged_stream = std::make_unique<SummingSortedBlockInputStream>(
src_streams, sort_desc, data.merging_params.columns_to_sum, DEFAULT_MERGE_BLOCK_SIZE);
src_streams, sort_description, data.merging_params.columns_to_sum, DEFAULT_MERGE_BLOCK_SIZE);
break;
case MergeTreeData::MergingParams::Aggregating:
merged_stream = std::make_unique<AggregatingSortedBlockInputStream>(
src_streams, sort_desc, DEFAULT_MERGE_BLOCK_SIZE);
src_streams, sort_description, DEFAULT_MERGE_BLOCK_SIZE);
break;
case MergeTreeData::MergingParams::Replacing:
merged_stream = std::make_unique<ReplacingSortedBlockInputStream>(
src_streams, sort_desc, data.merging_params.version_column, DEFAULT_MERGE_BLOCK_SIZE, rows_sources_write_buf.get());
src_streams, sort_description, data.merging_params.version_column, DEFAULT_MERGE_BLOCK_SIZE, rows_sources_write_buf.get());
break;
case MergeTreeData::MergingParams::Graphite:
merged_stream = std::make_unique<GraphiteRollupSortedBlockInputStream>(
src_streams, sort_desc, DEFAULT_MERGE_BLOCK_SIZE,
src_streams, sort_description, DEFAULT_MERGE_BLOCK_SIZE,
data.merging_params.graphite_params, time_of_merge);
break;
case MergeTreeData::MergingParams::VersionedCollapsing:
merged_stream = std::make_unique<VersionedCollapsingSortedBlockInputStream>(
src_streams, sort_desc, data.merging_params.sign_column, DEFAULT_MERGE_BLOCK_SIZE, false, rows_sources_write_buf.get());
src_streams, sort_description, data.merging_params.sign_column, DEFAULT_MERGE_BLOCK_SIZE, false, rows_sources_write_buf.get());
break;
default:
@ -927,12 +936,12 @@ MergeTreeData::DataPartPtr MergeTreeDataMergerMutator::renameMergedTemporaryPart
* When M > N parts could be replaced?
* - new block was added in ReplicatedMergeTreeBlockOutputStream;
* - it was added to working dataset in memory and renamed on filesystem;
* - but ZooKeeper transaction that add its to reference dataset in ZK and unlocks AbandonableLock is failed;
* - but ZooKeeper transaction that adds it to reference dataset in ZK failed;
* - and it is failed due to connection loss, so we don't rollback working dataset in memory,
* because we don't know if the part was added to ZK or not
* (see ReplicatedMergeTreeBlockOutputStream)
* - then method selectPartsToMerge selects a range and see, that AbandonableLock for this part is abandoned,
* and so, it is possible to merge a range skipping this part.
* - then method selectPartsToMerge selects a range and sees, that EphemeralLock for the block in this part is unlocked,
* and so it is possible to merge a range skipping this part.
* (NOTE: Merging with part that is not in ZK is not possible, see checks in 'createLogEntryToMergeParts'.)
* - and after merge, this part will be removed in addition to parts that was merged.
*/

View File

@ -71,7 +71,7 @@ public:
*/
bool selectAllPartsToMergeWithinPartition(
FuturePart & future_part,
size_t available_disk_space,
size_t & available_disk_space,
const AllowedMergingPredicate & can_merge,
const String & partition_id,
bool final,

View File

@ -448,7 +448,7 @@ void MergeTreeDataPart::loadIndex()
.getSize() / MERGE_TREE_MARK_SIZE;
}
size_t key_size = storage.primary_sort_descr.size();
size_t key_size = storage.primary_sort_columns.size();
if (key_size)
{
@ -630,7 +630,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata)
if (!checksums.empty())
{
if (!storage.primary_sort_descr.empty() && !checksums.files.count("primary.idx"))
if (!storage.primary_sort_columns.empty() && !checksums.files.count("primary.idx"))
throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART);
if (require_part_metadata)
@ -683,7 +683,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata)
};
/// Check that the primary key index is not empty.
if (!storage.primary_sort_descr.empty())
if (!storage.primary_sort_columns.empty())
check_file_not_empty(path + "primary.idx");
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)

View File

@ -196,17 +196,18 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
processed_stage = QueryProcessingStage::FetchColumns;
const Settings & settings = context.getSettingsRef();
SortDescription sort_descr = data.getPrimarySortDescription();
Names primary_sort_columns = data.getPrimarySortColumns();
KeyCondition key_condition(query_info, context, available_real_and_virtual_columns, sort_descr,
data.getPrimaryExpression());
KeyCondition key_condition(
query_info, context, available_real_and_virtual_columns,
primary_sort_columns, data.getPrimaryExpression());
if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue())
{
std::stringstream exception_message;
exception_message << "Primary key (";
for (size_t i = 0, size = sort_descr.size(); i < size; ++i)
exception_message << (i == 0 ? "" : ", ") << sort_descr[i].column_name;
for (size_t i = 0, size = primary_sort_columns.size(); i < size; ++i)
exception_message << (i == 0 ? "" : ", ") << primary_sort_columns[i];
exception_message << ") is not used and setting 'force_primary_key' is set.";
throw Exception(exception_message.str(), ErrorCodes::INDEX_NOT_USED);
@ -217,7 +218,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
{
minmax_idx_condition.emplace(
query_info, context, available_real_and_virtual_columns,
data.minmax_idx_sort_descr, data.minmax_idx_expr);
data.minmax_idx_columns, data.minmax_idx_expr);
if (settings.force_index_by_date && minmax_idx_condition->alwaysUnknownOrTrue())
{
@ -781,36 +782,44 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal
to_merge.emplace_back(std::make_shared<ExpressionBlockInputStream>(source_stream, data.getPrimaryExpression()));
}
BlockInputStreamPtr merged;
Names sort_columns = data.getSortColumns();
SortDescription sort_description;
size_t sort_columns_size = sort_columns.size();
sort_description.reserve(sort_columns_size);
Block header = to_merge.at(0)->getHeader();
for (size_t i = 0; i < sort_columns_size; ++i)
sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1);
BlockInputStreamPtr merged;
switch (data.merging_params.mode)
{
case MergeTreeData::MergingParams::Ordinary:
merged = std::make_shared<MergingSortedBlockInputStream>(to_merge, data.getSortDescription(), max_block_size);
merged = std::make_shared<MergingSortedBlockInputStream>(to_merge, sort_description, max_block_size);
break;
case MergeTreeData::MergingParams::Collapsing:
merged = std::make_shared<CollapsingFinalBlockInputStream>(
to_merge, data.getSortDescription(), data.merging_params.sign_column);
to_merge, sort_description, data.merging_params.sign_column);
break;
case MergeTreeData::MergingParams::Summing:
merged = std::make_shared<SummingSortedBlockInputStream>(to_merge,
data.getSortDescription(), data.merging_params.columns_to_sum, max_block_size);
sort_description, data.merging_params.columns_to_sum, max_block_size);
break;
case MergeTreeData::MergingParams::Aggregating:
merged = std::make_shared<AggregatingSortedBlockInputStream>(to_merge, data.getSortDescription(), max_block_size);
merged = std::make_shared<AggregatingSortedBlockInputStream>(to_merge, sort_description, max_block_size);
break;
case MergeTreeData::MergingParams::Replacing: /// TODO Make ReplacingFinalBlockInputStream
merged = std::make_shared<ReplacingSortedBlockInputStream>(to_merge,
data.getSortDescription(), data.merging_params.version_column, max_block_size);
sort_description, data.merging_params.version_column, max_block_size);
break;
case MergeTreeData::MergingParams::VersionedCollapsing: /// TODO Make VersionedCollapsingFinalBlockInputStream
merged = std::make_shared<VersionedCollapsingSortedBlockInputStream>(
to_merge, data.getSortDescription(), data.merging_params.sign_column, max_block_size, true);
to_merge, sort_description, data.merging_params.sign_column, max_block_size, true);
break;
case MergeTreeData::MergingParams::Graphite:

View File

@ -183,7 +183,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
secondary_sort_expr->execute(block);
}
SortDescription sort_descr = data.getSortDescription();
Names sort_columns = data.getSortColumns();
SortDescription sort_description;
size_t sort_columns_size = sort_columns.size();
sort_description.reserve(sort_columns_size);
for (size_t i = 0; i < sort_columns_size; ++i)
sort_description.emplace_back(block.getPositionByName(sort_columns[i]), 1, 1);
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocks);
@ -192,9 +198,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
IColumn::Permutation perm;
if (data.hasPrimaryKey())
{
if (!isAlreadySorted(block, sort_descr))
if (!isAlreadySorted(block, sort_description))
{
stableGetPermutation(block, sort_descr, perm);
stableGetPermutation(block, sort_description, perm);
perm_ptr = &perm;
}
else

View File

@ -12,6 +12,7 @@
#include <Common/typeid_cast.h>
#include <DataTypes/NestedUtils.h>
#include <ext/scope_guard.h>
#include <ext/collection_cast.h>
#include <ext/map.h>
#include <memory>
#include <unordered_map>
@ -39,8 +40,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
const MergeTreeData & data,
const Names & column_names,
Logger * log)
: primary_key_columns{ext::map<std::unordered_set>(data.getPrimarySortDescription(),
[] (const SortColumnDescription & col) { return col.column_name; })},
: primary_key_columns{ext::collection_cast<std::unordered_set>(data.getPrimarySortColumns())},
table_columns{ext::map<std::unordered_set>(data.getColumns().getAllPhysical(),
[] (const NameAndTypePair & col) { return col.name; })},
block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, context, data.getColumns().getAllPhysical())},

View File

@ -365,26 +365,20 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
/// The set of written offset columns so that you do not write shared offsets of nested structures columns several times
OffsetColumns offset_columns;
auto sort_description = storage.getPrimarySortDescription();
auto sort_columns = storage.getPrimarySortColumns();
/// Here we will add the columns related to the Primary Key, then write the index.
std::vector<ColumnWithTypeAndName> primary_columns(sort_description.size());
std::vector<ColumnWithTypeAndName> primary_columns(sort_columns.size());
std::map<String, size_t> primary_columns_name_to_position;
for (size_t i = 0, size = sort_description.size(); i < size; ++i)
for (size_t i = 0, size = sort_columns.size(); i < size; ++i)
{
const auto & descr = sort_description[i];
String name = !descr.column_name.empty()
? descr.column_name
: block.safeGetByPosition(descr.column_number).name;
const auto & name = sort_columns[i];
if (!primary_columns_name_to_position.emplace(name, i).second)
throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS);
primary_columns[i] = !descr.column_name.empty()
? block.getByName(descr.column_name)
: block.safeGetByPosition(descr.column_number);
primary_columns[i] = block.getByName(name);
/// Reorder primary key columns in advance and add them to `primary_columns`.
if (permutation)
@ -393,8 +387,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
if (index_columns.empty())
{
index_columns.resize(sort_description.size());
for (size_t i = 0, size = sort_description.size(); i < size; ++i)
index_columns.resize(sort_columns.size());
for (size_t i = 0, size = sort_columns.size(); i < size; ++i)
index_columns[i] = primary_columns[i].column->cloneEmpty();
}

View File

@ -1,5 +1,5 @@
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/AbandonableLockInZooKeeper.h>
#include <Storages/MergeTree/EphemeralLockInZooKeeper.h>
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
#include <Interpreters/PartLog.h>

View File

@ -1266,15 +1266,19 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
prev_virtual_parts = queue.virtual_parts;
}
/// Load current quorum status.
auto quorum_last_part_future = zookeeper->asyncTryGet(queue.zookeeper_path + "/quorum/last_part");
auto quorum_status_future = zookeeper->asyncTryGet(queue.zookeeper_path + "/quorum/status");
/// Load current inserts
std::unordered_set<String> abandonable_lock_holders;
std::unordered_set<String> lock_holder_paths;
for (const String & entry : zookeeper->getChildren(queue.zookeeper_path + "/temp"))
{
if (startsWith(entry, "abandonable_lock-"))
abandonable_lock_holders.insert(queue.zookeeper_path + "/temp/" + entry);
lock_holder_paths.insert(queue.zookeeper_path + "/temp/" + entry);
}
if (!abandonable_lock_holders.empty())
if (!lock_holder_paths.empty())
{
Strings partitions = zookeeper->getChildren(queue.zookeeper_path + "/block_numbers");
std::vector<std::future<zkutil::ListResponse>> lock_futures;
@ -1310,21 +1314,22 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
for (BlockInfo & block : block_infos)
{
zkutil::GetResponse resp = block.contents_future.get();
if (!resp.error && abandonable_lock_holders.count(resp.data))
if (!resp.error && lock_holder_paths.count(resp.data))
committing_blocks[block.partition].insert(block.number);
}
}
queue_.pullLogsToQueue(zookeeper);
/// Load current quorum status.
zookeeper->tryGet(queue.zookeeper_path + "/quorum/last_part", last_quorum_part);
zkutil::GetResponse quorum_last_part_response = quorum_last_part_future.get();
if (!quorum_last_part_response.error)
last_quorum_part = quorum_last_part_response.data;
String quorum_status_str;
if (zookeeper->tryGet(queue.zookeeper_path + "/quorum/status", quorum_status_str))
zkutil::GetResponse quorum_status_response = quorum_status_future.get();
if (!quorum_status_response.error)
{
ReplicatedMergeTreeQuorumEntry quorum_status;
quorum_status.fromString(quorum_status_str);
quorum_status.fromString(quorum_status_response.data);
inprogress_quorum_part = quorum_status.part_name;
}
else
@ -1338,7 +1343,7 @@ bool ReplicatedMergeTreeMergePredicate::operator()(
/// A sketch of a proof of why this method actually works:
///
/// The trickiest part is to ensure that no new parts will ever appear in the range of blocks between left and right.
/// Inserted parts get their block numbers by acquiring an abandonable lock (see AbandonableLockInZooKeeper.h).
/// Inserted parts get their block numbers by acquiring an ephemeral lock (see EphemeralLockInZooKeeper.h).
/// These block numbers are monotonically increasing in a partition.
///
/// Because there is a window between the moment the inserted part gets its block number and

View File

@ -45,6 +45,11 @@ void MutationCommands::validate(const IStorage & table, const Context & context)
case MutationCommand::DELETE:
{
auto actions = ExpressionAnalyzer(command.predicate, context, {}, all_columns).getActions(true);
/// Try executing the resulting actions on the table sample block to detect malformed queries.
auto table_sample_block = table.getSampleBlock();
actions->execute(table_sample_block);
const ColumnWithTypeAndName & predicate_column = actions->getSampleBlock().getByName(
command.predicate->getColumnName());
checkColumnCanBeUsedAsFilter(predicate_column);

View File

@ -3,8 +3,9 @@
#include <Databases/IDatabase.h>
#include <DataTypes/DataTypeFactory.h>
#include <Storages/StorageDistributed.h>
#include <Storages/VirtualColumnFactory.h>
#include <Storages/Distributed/DistributedBlockOutputStream.h>
#include <Storages/Distributed/DirectoryMonitor.h>
#include <Storages/StorageFactory.h>
@ -57,6 +58,7 @@ namespace ErrorCodes
extern const int INCORRECT_NUMBER_OF_COLUMNS;
extern const int INFINITE_LOOP;
extern const int TYPE_MISMATCH;
extern const int NO_SUCH_COLUMN_IN_TABLE;
}
@ -319,18 +321,36 @@ void StorageDistributed::truncate(const ASTPtr &)
}
}
namespace
{
/// NOTE This is weird. Get rid of this.
std::map<String, String> virtual_columns =
{
{"_table", "String"},
{"_part", "String"},
{"_part_index", "UInt64"},
{"_sample_factor", "Float64"},
};
}
NameAndTypePair StorageDistributed::getColumn(const String & column_name) const
{
if (const auto & type = VirtualColumnFactory::tryGetType(column_name))
return { column_name, type };
if (getColumns().hasPhysical(column_name))
return getColumns().getPhysical(column_name);
return getColumns().getPhysical(column_name);
auto it = virtual_columns.find(column_name);
if (it != virtual_columns.end())
return { it->first, DataTypeFactory::instance().get(it->second) };
throw Exception("There is no column " + column_name + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
}
bool StorageDistributed::hasColumn(const String & column_name) const
{
return VirtualColumnFactory::hasColumn(column_name) || getColumns().hasPhysical(column_name);
return virtual_columns.count(column_name) || getColumns().hasPhysical(column_name);
}
void StorageDistributed::createDirectoryMonitors()

View File

@ -11,8 +11,6 @@
#include <Storages/StorageMaterializedView.h>
#include <Storages/StorageFactory.h>
#include <Storages/VirtualColumnFactory.h>
#include <Common/typeid_cast.h>

View File

@ -9,7 +9,6 @@
#include <Storages/StorageMerge.h>
#include <Storages/StorageFactory.h>
#include <Storages/VirtualColumnUtils.h>
#include <Storages/VirtualColumnFactory.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/evaluateConstantExpression.h>
@ -32,6 +31,7 @@ namespace ErrorCodes
extern const int ILLEGAL_PREWHERE;
extern const int INCOMPATIBLE_SOURCE_TABLES;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NO_SUCH_COLUMN_IN_TABLE;
}
@ -48,22 +48,44 @@ StorageMerge::StorageMerge(
}
/// NOTE Structure of underlying tables as well as their set are not constant,
/// so the results of these methods may become obsolete after the call.
NameAndTypePair StorageMerge::getColumn(const String & column_name) const
{
auto type = VirtualColumnFactory::tryGetType(column_name);
if (type)
return NameAndTypePair(column_name, type);
/// virtual column of the Merge table itself
if (column_name == "_table")
return { column_name, std::make_shared<DataTypeString>() };
return IStorage::getColumn(column_name);
if (IStorage::hasColumn(column_name))
return IStorage::getColumn(column_name);
/// virtual (and real) columns of the underlying tables
auto first_table = getFirstTable([](auto &&) { return true; });
if (first_table)
return first_table->getColumn(column_name);
throw Exception("There is no column " + column_name + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
}
bool StorageMerge::hasColumn(const String & column_name) const
{
return VirtualColumnFactory::hasColumn(column_name) || IStorage::hasColumn(column_name);
if (column_name == "_table")
return true;
if (IStorage::hasColumn(column_name))
return true;
auto first_table = getFirstTable([](auto &&) { return true; });
if (first_table)
return first_table->hasColumn(column_name);
return false;
}
bool StorageMerge::isRemote() const
template <typename F>
StoragePtr StorageMerge::getFirstTable(F && predicate) const
{
auto database = context.getDatabase(source_database);
auto iterator = database->getIterator(context);
@ -73,14 +95,21 @@ bool StorageMerge::isRemote() const
if (table_name_regexp.match(iterator->name()))
{
auto & table = iterator->table();
if (table.get() != this && table->isRemote())
return true;
if (table.get() != this && predicate(table))
return table;
}
iterator->next();
}
return false;
return {};
}
bool StorageMerge::isRemote() const
{
auto first_remote_table = getFirstTable([](const StoragePtr & table) { return table->isRemote(); });
return first_remote_table != nullptr;
}

View File

@ -58,6 +58,9 @@ private:
Block getBlockWithVirtualColumns(const StorageListWithLocks & selected_tables) const;
template <typename F>
StoragePtr getFirstTable(F && predicate) const;
protected:
StorageMerge(
const std::string & name_,

View File

@ -479,11 +479,32 @@ bool StorageMergeTree::optimize(
partition_id = data.getPartitionIDFromQuery(partition, context);
String disable_reason;
if (!merge(context.getSettingsRef().min_bytes_to_use_direct_io, true, partition_id, final, deduplicate, &disable_reason))
if (!partition && final)
{
if (context.getSettingsRef().optimize_throw_if_noop)
throw Exception(disable_reason.empty() ? "Can't OPTIMIZE by some reason" : disable_reason, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
return false;
MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector();
std::unordered_set<String> partition_ids;
for (const MergeTreeData::DataPartPtr & part : data_parts)
partition_ids.emplace(part->info.partition_id);
for (const String & partition_id : partition_ids)
{
if (!merge(context.getSettingsRef().min_bytes_to_use_direct_io, true, partition_id, true, deduplicate, &disable_reason))
{
if (context.getSettingsRef().optimize_throw_if_noop)
throw Exception(disable_reason.empty() ? "Can't OPTIMIZE by some reason" : disable_reason, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
return false;
}
}
}
else
{
if (!merge(context.getSettingsRef().min_bytes_to_use_direct_io, true, partition_id, final, deduplicate, &disable_reason))
{
if (context.getSettingsRef().optimize_throw_if_noop)
throw Exception(disable_reason.empty() ? "Can't OPTIMIZE by some reason" : disable_reason, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
return false;
}
}
return true;

View File

@ -2062,7 +2062,10 @@ void StorageReplicatedMergeTree::queueUpdatingTask()
tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
{
restarting_thread->wakeup();
return;
}
queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
}
@ -2279,14 +2282,20 @@ bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
bool deduplicate,
ReplicatedMergeTreeLogEntryData * out_log_entry)
{
bool all_in_zk = true;
std::vector<std::future<zkutil::ExistsResponse>> exists_futures;
exists_futures.reserve(parts.size());
for (const auto & part : parts)
exists_futures.emplace_back(zookeeper->asyncExists(replica_path + "/parts/" + part->name));
bool all_in_zk = true;
for (size_t i = 0; i < parts.size(); ++i)
{
/// If there is no information about part in ZK, we will not merge it.
if (!zookeeper->exists(replica_path + "/parts/" + part->name))
if (exists_futures[i].get().error == ZooKeeperImpl::ZooKeeper::ZNONODE)
{
all_in_zk = false;
const auto & part = parts[i];
if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(nullptr))
{
LOG_WARNING(log, "Part " << part->name << " (that was selected for merge)"
@ -2297,6 +2306,7 @@ bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
}
}
}
if (!all_in_zk)
return false;
@ -2313,16 +2323,6 @@ bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
String path_created = zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1);
const String & partition_id = parts[0]->info.partition_id;
for (size_t i = 0; i + 1 < parts.size(); ++i)
{
/// Remove the unnecessary entries about non-existent blocks.
for (Int64 number = parts[i]->info.max_block + 1; number <= parts[i + 1]->info.min_block - 1; ++number)
{
zookeeper->tryRemove(zookeeper_path + "/block_numbers/" + partition_id + "/block-" + padIndex(number));
}
}
if (out_log_entry)
*out_log_entry = entry;
@ -2904,26 +2904,9 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
MergeTreeDataMergerMutator::FuturePart future_merged_part;
String disable_reason;
bool selected = false;
auto zookeeper = getZooKeeper();
ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper);
if (!partition)
{
selected = merger_mutator.selectPartsToMerge(
future_merged_part, true, data.settings.max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason);
}
else
{
String partition_id = data.getPartitionIDFromQuery(partition, context);
selected = merger_mutator.selectAllPartsToMergeWithinPartition(
future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason);
}
auto handle_noop = [&] (const String & message)
{
if (context.getSettingsRef().optimize_throw_if_noop)
@ -2931,14 +2914,50 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
return false;
};
if (!selected)
if (!partition && final)
{
LOG_INFO(log, "Cannot select parts for optimization" + (disable_reason.empty() ? "" : ": " + disable_reason));
return handle_noop(disable_reason);
}
MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector();
std::unordered_set<String> partition_ids;
if (!createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, deduplicate, &merge_entry))
return handle_noop("Can't create merge queue node in ZooKeeper");
for (const MergeTreeData::DataPartPtr & part : data_parts)
partition_ids.emplace(part->info.partition_id);
for (const String & partition_id : partition_ids)
{
MergeTreeDataMergerMutator::FuturePart future_merged_part;
bool selected = merger_mutator.selectAllPartsToMergeWithinPartition(
future_merged_part, disk_space, can_merge, partition_id, true, nullptr);
if (selected &&
!createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, deduplicate, &merge_entry))
return handle_noop("Can't create merge queue node in ZooKeeper");
}
}
else
{
MergeTreeDataMergerMutator::FuturePart future_merged_part;
String disable_reason;
bool selected = false;
if (!partition)
{
selected = merger_mutator.selectPartsToMerge(
future_merged_part, true, data.settings.max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason);
}
else
{
String partition_id = data.getPartitionIDFromQuery(partition, context);
selected = merger_mutator.selectAllPartsToMergeWithinPartition(
future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason);
}
if (!selected)
{
LOG_INFO(log, "Cannot select parts for optimization" + (disable_reason.empty() ? "" : ": " + disable_reason));
return handle_noop(disable_reason);
}
if (!createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, deduplicate, &merge_entry))
return handle_noop("Can't create merge queue node in ZooKeeper");
}
}
/// TODO: Bad setting name for such purpose
@ -3404,7 +3423,7 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path)
}
std::optional<AbandonableLockInZooKeeper>
std::optional<EphemeralLockInZooKeeper>
StorageReplicatedMergeTree::allocateBlockNumber(
const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path)
{
@ -3434,11 +3453,11 @@ StorageReplicatedMergeTree::allocateBlockNumber(
zkutil::KeeperMultiException::check(code, ops, responses);
}
AbandonableLockInZooKeeper lock;
EphemeralLockInZooKeeper lock;
/// 2 RTT
try
{
lock = AbandonableLockInZooKeeper(
lock = EphemeralLockInZooKeeper(
partition_path + "/block-", zookeeper_path + "/temp", *zookeeper, &deduplication_check_ops);
}
catch (const zkutil::KeeperMultiException & e)
@ -4375,7 +4394,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
MergeTreeData::MutableDataPartsVector dst_parts;
Strings block_id_paths;
Strings part_checksums;
std::vector<AbandonableLockInZooKeeper> abandonable_locks;
std::vector<EphemeralLockInZooKeeper> ephemeral_locks;
LOG_DEBUG(log, "Cloning " << src_all_parts.size() << " parts");
@ -4431,7 +4450,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
src_parts.emplace_back(src_part);
dst_parts.emplace_back(dst_part);
abandonable_locks.emplace_back(std::move(*lock));
ephemeral_locks.emplace_back(std::move(*lock));
block_id_paths.emplace_back(block_id_path);
part_checksums.emplace_back(hash_hex);
}
@ -4472,7 +4491,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
for (size_t i = 0; i < dst_parts.size(); ++i)
{
getCommitPartOps(ops, dst_parts[i], block_id_paths[i]);
abandonable_locks[i].getUnlockOps(ops);
ephemeral_locks[i].getUnlockOps(ops);
if (ops.size() > zkutil::MULTI_BATCH_SIZE)
{
@ -4513,7 +4532,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
String log_znode_path = dynamic_cast<const zkutil::CreateResponse &>(*op_results.back()).path_created;
entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
for (auto & lock : abandonable_locks)
for (auto & lock : ephemeral_locks)
lock.assumeUnlocked();
/// Forcibly remove replaced parts from ZooKeeper

View File

@ -14,7 +14,7 @@
#include <Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
#include <Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h>
#include <Storages/MergeTree/ReplicatedMergeTreeAlterThread.h>
#include <Storages/MergeTree/AbandonableLockInZooKeeper.h>
#include <Storages/MergeTree/EphemeralLockInZooKeeper.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <Storages/MergeTree/DataPartsExchange.h>
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
@ -460,8 +460,9 @@ private:
void updateQuorum(const String & part_name);
/// Creates new block number if block with such block_id does not exist
std::optional<AbandonableLockInZooKeeper> allocateBlockNumber(const String & partition_id, zkutil::ZooKeeperPtr & zookeeper,
const String & zookeeper_block_id_path = "");
std::optional<EphemeralLockInZooKeeper> allocateBlockNumber(
const String & partition_id, zkutil::ZooKeeperPtr & zookeeper,
const String & zookeeper_block_id_path = "");
/** Wait until all replicas, including this, execute the specified action from the log.
* If replicas are added at the same time, it can not wait the added replica .

View File

@ -105,7 +105,7 @@ StorageSet::StorageSet(
const String & name_,
const ColumnsDescription & columns_)
: StorageSetOrJoinBase{path_, name_, columns_},
set(std::make_shared<Set>(SizeLimits()))
set(std::make_shared<Set>(SizeLimits(), false))
{
Block header = getSampleBlock();
header = header.sortColumns();
@ -115,7 +115,7 @@ StorageSet::StorageSet(
}
void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block, /*fill_set_elements=*/false); }
void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block); }
size_t StorageSet::getSize() const { return set->getTotalRowCount(); }
@ -129,7 +129,7 @@ void StorageSet::truncate(const ASTPtr &)
header = header.sortColumns();
increment = 0;
set = std::make_shared<Set>(SizeLimits());
set = std::make_shared<Set>(SizeLimits(), false);
set->setHeader(header);
};

View File

@ -245,6 +245,10 @@ void TinyLogBlockOutputStream::writeSuffix()
return;
done = true;
/// If nothing was written - leave the table in initial state.
if (streams.empty())
return;
/// Finish write.
for (auto & stream : streams)
stream.second->finalize();

View File

@ -22,95 +22,6 @@ namespace ErrorCodes
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
}
/// Some virtual columns routines
namespace
{
bool hasColumn(const ColumnsWithTypeAndName & columns, const String & column_name)
{
for (const auto & column : columns)
{
if (column.name == column_name)
return true;
}
return false;
}
NameAndTypePair tryGetColumn(const ColumnsWithTypeAndName & columns, const String & column_name)
{
for (const auto & column : columns)
{
if (column.name == column_name)
return {column.name, column.type};
}
return {};
}
struct VirtualColumnsProcessor
{
explicit VirtualColumnsProcessor(const ColumnsWithTypeAndName & all_virtual_columns_)
: all_virtual_columns(all_virtual_columns_), virtual_columns_mask(all_virtual_columns_.size(), 0) {}
/// Separates real and virtual column names, returns real ones
Names process(const Names & column_names, const std::vector<bool *> & virtual_columns_exists_flag = {})
{
Names real_column_names;
if (!virtual_columns_exists_flag.empty())
{
for (size_t i = 0; i < all_virtual_columns.size(); ++i)
*virtual_columns_exists_flag.at(i) = false;
}
for (const String & column_name : column_names)
{
ssize_t virtual_column_index = -1;
for (size_t i = 0; i < all_virtual_columns.size(); ++i)
{
if (column_name == all_virtual_columns[i].name)
{
virtual_column_index = i;
break;
}
}
if (virtual_column_index >= 0)
{
auto index = static_cast<size_t>(virtual_column_index);
virtual_columns_mask[index] = 1;
if (!virtual_columns_exists_flag.empty())
*virtual_columns_exists_flag.at(index) = true;
}
else
{
real_column_names.emplace_back(column_name);
}
}
return real_column_names;
}
void appendVirtualColumns(Block & block)
{
for (size_t i = 0; i < all_virtual_columns.size(); ++i)
{
if (virtual_columns_mask[i])
block.insert(all_virtual_columns[i].cloneEmpty());
}
}
protected:
const ColumnsWithTypeAndName & all_virtual_columns;
std::vector<UInt8> virtual_columns_mask;
};
}
StorageSystemTables::StorageSystemTables(const std::string & name_)
: name(name_)
@ -123,14 +34,10 @@ StorageSystemTables::StorageSystemTables(const std::string & name_)
{"is_temporary", std::make_shared<DataTypeUInt8>()},
{"data_path", std::make_shared<DataTypeString>()},
{"metadata_path", std::make_shared<DataTypeString>()},
{"metadata_modification_time", std::make_shared<DataTypeDateTime>()},
{"create_table_query", std::make_shared<DataTypeString>()},
{"engine_full", std::make_shared<DataTypeString>()}
}));
virtual_columns =
{
{std::make_shared<DataTypeDateTime>(), "metadata_modification_time"},
{std::make_shared<DataTypeString>(), "create_table_query"},
{std::make_shared<DataTypeString>(), "engine_full"}
};
}
@ -156,17 +63,24 @@ BlockInputStreams StorageSystemTables::read(
{
processed_stage = QueryProcessingStage::FetchColumns;
Names real_column_names;
bool has_metadata_modification_time = false;
bool has_create_table_query = false;
bool has_engine_full = false;
check(column_names);
VirtualColumnsProcessor virtual_columns_processor(virtual_columns);
real_column_names = virtual_columns_processor.process(column_names, {&has_metadata_modification_time, &has_create_table_query, &has_engine_full});
check(real_column_names);
/// Create a mask of what columns are needed in the result.
Block res_block = getSampleBlock();
virtual_columns_processor.appendVirtualColumns(res_block);
NameSet names_set(column_names.begin(), column_names.end());
Block sample_block = getSampleBlock();
Block res_block;
std::vector<UInt8> columns_mask(sample_block.columns());
for (size_t i = 0, size = columns_mask.size(); i < size; ++i)
{
if (names_set.count(sample_block.getByPosition(i).name))
{
columns_mask[i] = 1;
res_block.insert(sample_block.getByPosition(i));
}
}
MutableColumns res_columns = res_block.cloneEmptyColumns();
@ -188,25 +102,38 @@ BlockInputStreams StorageSystemTables::read(
{
auto table_name = iterator->name();
size_t j = 0;
res_columns[j++]->insert(database_name);
res_columns[j++]->insert(table_name);
res_columns[j++]->insert(iterator->table()->getName());
res_columns[j++]->insert(UInt64(0));
res_columns[j++]->insert(iterator->table()->getDataPath());
res_columns[j++]->insert(database->getTableMetadataPath(table_name));
size_t src_index = 0;
size_t res_index = 0;
if (has_metadata_modification_time)
res_columns[j++]->insert(static_cast<UInt64>(database->getTableMetadataModificationTime(context, table_name)));
if (columns_mask[src_index++])
res_columns[res_index++]->insert(database_name);
if (has_create_table_query || has_engine_full)
if (columns_mask[src_index++])
res_columns[res_index++]->insert(table_name);
if (columns_mask[src_index++])
res_columns[res_index++]->insert(iterator->table()->getName());
if (columns_mask[src_index++])
res_columns[res_index++]->insert(UInt64(0));
if (columns_mask[src_index++])
res_columns[res_index++]->insert(iterator->table()->getDataPath());
if (columns_mask[src_index++])
res_columns[res_index++]->insert(database->getTableMetadataPath(table_name));
if (columns_mask[src_index++])
res_columns[res_index++]->insert(static_cast<UInt64>(database->getTableMetadataModificationTime(context, table_name)));
if (columns_mask[src_index] || columns_mask[src_index + 1])
{
ASTPtr ast = database->tryGetCreateTableQuery(context, table_name);
if (has_create_table_query)
res_columns[j++]->insert(ast ? queryToString(ast) : "");
if (columns_mask[src_index++])
res_columns[res_index++]->insert(ast ? queryToString(ast) : "");
if (has_engine_full)
if (columns_mask[src_index++])
{
String engine_full;
@ -223,34 +150,49 @@ BlockInputStreams StorageSystemTables::read(
}
}
res_columns[j++]->insert(engine_full);
res_columns[res_index++]->insert(engine_full);
}
}
}
}
/// This is for temporary tables.
if (context.hasSessionContext())
{
Tables external_tables = context.getSessionContext().getExternalTables();
for (auto table : external_tables)
{
size_t j = 0;
res_columns[j++]->insertDefault();
res_columns[j++]->insert(table.first);
res_columns[j++]->insert(table.second->getName());
res_columns[j++]->insert(UInt64(1));
res_columns[j++]->insertDefault();
res_columns[j++]->insertDefault();
size_t src_index = 0;
size_t res_index = 0;
if (has_metadata_modification_time)
res_columns[j++]->insertDefault();
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
if (has_create_table_query)
res_columns[j++]->insertDefault();
if (columns_mask[src_index++])
res_columns[res_index++]->insert(table.first);
if (has_engine_full)
res_columns[j++]->insert(table.second->getName());
if (columns_mask[src_index++])
res_columns[res_index++]->insert(table.second->getName());
if (columns_mask[src_index++])
res_columns[res_index++]->insert(UInt64(1));
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
if (columns_mask[src_index++])
res_columns[res_index++]->insert(table.second->getName());
}
}
@ -258,15 +200,4 @@ BlockInputStreams StorageSystemTables::read(
return {std::make_shared<OneBlockInputStream>(res_block)};
}
bool StorageSystemTables::hasColumn(const String & column_name) const
{
return DB::hasColumn(virtual_columns, column_name) || ITableDeclaration::hasColumn(column_name);
}
NameAndTypePair StorageSystemTables::getColumn(const String & column_name) const
{
auto virtual_column = DB::tryGetColumn(virtual_columns, column_name);
return !virtual_column.name.empty() ? virtual_column : ITableDeclaration::getColumn(column_name);
}
}

View File

@ -26,15 +26,9 @@ public:
size_t max_block_size,
unsigned num_streams) override;
bool hasColumn(const String & column_name) const override;
NameAndTypePair getColumn(const String & column_name) const override;
private:
const std::string name;
ColumnsWithTypeAndName virtual_columns;
protected:
StorageSystemTables(const std::string & name_);
};

View File

@ -1,37 +0,0 @@
#include <Storages/VirtualColumnFactory.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NO_SUCH_COLUMN_IN_TABLE;
}
DataTypePtr VirtualColumnFactory::getType(const String & name)
{
auto res = tryGetType(name);
if (!res)
throw Exception("There is no column " + name + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
return res;
}
bool VirtualColumnFactory::hasColumn(const String & name)
{
return !!tryGetType(name);
}
DataTypePtr VirtualColumnFactory::tryGetType(const String & name)
{
if (name == "_table") return std::make_shared<DataTypeString>();
if (name == "_part") return std::make_shared<DataTypeString>();
if (name == "_part_index") return std::make_shared<DataTypeUInt64>();
if (name == "_sample_factor") return std::make_shared<DataTypeFloat64>();
return nullptr;
}
}

View File

@ -1,20 +0,0 @@
#pragma once
#include <DataTypes/IDataType.h>
namespace DB
{
/** Knows the names and types of all possible virtual columns.
* It is necessary for engines that redirect a request to other tables without knowing in advance what virtual columns they contain.
*/
class VirtualColumnFactory
{
public:
static bool hasColumn(const String & name);
static DataTypePtr getType(const String & name);
static DataTypePtr tryGetType(const String & name);
};
}

View File

@ -4,7 +4,7 @@
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/Exception.h>
#include <Common/Stopwatch.h>
#include <Storages/MergeTree/AbandonableLockInZooKeeper.h>
#include <Storages/MergeTree/EphemeralLockInZooKeeper.h>
#include <ext/scope_guard.h>
#include <pcg_random.hpp>

View File

@ -4,7 +4,7 @@
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/Exception.h>
#include <Common/Stopwatch.h>
#include <Storages/MergeTree/AbandonableLockInZooKeeper.h>
#include <Storages/MergeTree/EphemeralLockInZooKeeper.h>
#include <ext/scope_guard.h>
#include <pcg_random.hpp>
@ -37,17 +37,17 @@ try
Stopwatch total;
Stopwatch stage;
/// Load current inserts
std::unordered_set<String> abandonable_lock_holders;
std::unordered_set<String> lock_holder_paths;
for (const String & entry : zookeeper->getChildren(zookeeper_path + "/temp"))
{
if (startsWith(entry, "abandonable_lock-"))
abandonable_lock_holders.insert(zookeeper_path + "/temp/" + entry);
lock_holder_paths.insert(zookeeper_path + "/temp/" + entry);
}
std::cerr << "Stage 1 (get lock holders): " << abandonable_lock_holders.size()
std::cerr << "Stage 1 (get lock holders): " << lock_holder_paths.size()
<< " lock holders, elapsed: " << stage.elapsedSeconds() << "s." << std::endl;
stage.restart();
if (!abandonable_lock_holders.empty())
if (!lock_holder_paths.empty())
{
Strings partitions = zookeeper->getChildren(zookeeper_path + "/block_numbers");
std::cerr << "Stage 2 (get partitions): " << partitions.size()
@ -86,7 +86,7 @@ try
for (BlockInfo & block : block_infos)
{
zkutil::GetResponse resp = block.contents_future.get();
if (!resp.error && abandonable_lock_holders.count(resp.data))
if (!resp.error && lock_holder_paths.count(resp.data))
{
++total_count;
current_inserts[block.partition].insert(block.number);

View File

@ -26,8 +26,7 @@ try
names_and_types.emplace_back("a", std::make_shared<DataTypeUInt64>());
names_and_types.emplace_back("b", std::make_shared<DataTypeUInt8>());
StoragePtr table = StorageLog::create(
"./", "test", ColumnsDescription{names_and_types}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
StoragePtr table = StorageLog::create("./", "test", ColumnsDescription{names_and_types}, 1048576);
table->startup();
/// write into it

View File

@ -1,6 +1,5 @@
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/ITableFunctionFileLike.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Common/Exception.h>

View File

@ -6,8 +6,8 @@ target_include_directories (dictionary_library_c PRIVATE ${DBMS_INCLUDE_DIR})
add_library (dictionary_library_empty SHARED "dictionary_library_empty.cpp")
target_include_directories (dictionary_library_empty PRIVATE ${DBMS_INCLUDE_DIR})
# Don't change lib name in debug build
# Don't add "lib" prefix, and don't change lib name in debug build
# because result .so will be pointed in dictionary_*.xml
set_target_properties(dictionary_library PROPERTIES DEBUG_POSTFIX "")
set_target_properties(dictionary_library_c PROPERTIES DEBUG_POSTFIX "")
set_target_properties(dictionary_library_empty PROPERTIES DEBUG_POSTFIX "")
set_target_properties(dictionary_library PROPERTIES PREFIX "" DEBUG_POSTFIX "")
set_target_properties(dictionary_library_c PROPERTIES PREFIX "" DEBUG_POSTFIX "")
set_target_properties(dictionary_library_empty PROPERTIES PREFIX "" DEBUG_POSTFIX "")

View File

@ -396,14 +396,14 @@ def generate_dictionaries(args):
<library>
<path>{filename}</path>
</library>
'''.format(filename=os.path.abspath('../../../build/dbms/tests/external_dictionaries/dictionary_library/libdictionary_library.so'))
'''.format(filename=os.path.abspath('../../../build/dbms/tests/external_dictionaries/dictionary_library/dictionary_library.so'))
# Todo?
#source_library_c = '''
#<library>
# <path>{filename}</path>
#</library>
#'''.format(filename=os.path.abspath('../../../build/dbms/tests/external_dictionaries/dict_lib/libdict_library_c.so'))
#'''.format(filename=os.path.abspath('../../../build/dbms/tests/external_dictionaries/dictionary_library/dictionary_library_c.so'))
layout_flat = '<flat />'

View File

@ -134,15 +134,19 @@ def test_insert_multithreaded(started_cluster):
# Sanity check: at least something was inserted
assert runner.total_inserted > 0
for i in range(30): # wait for replication 3 seconds max
all_replicated = False
for i in range(50): # wait for replication 5 seconds max
time.sleep(0.1)
def get_delay(node):
return int(node.query("SELECT absolute_delay FROM system.replicas WHERE table = 'repl_test'").rstrip())
if all([get_delay(n) == 0 for n in nodes]):
all_replicated = True
break
assert all_replicated
actual_inserted = []
for i, node in enumerate(nodes):
actual_inserted.append(int(node.query("SELECT sum(x) FROM repl_test").rstrip()))

View File

@ -27,7 +27,8 @@ def test_chroot_with_same_root():
CREATE TABLE simple (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192);
'''.format(replica=node.name))
node.query("INSERT INTO simple VALUES ({0}, {0})".format(i))
for j in range(2): # Second insert to test deduplication
node.query("INSERT INTO simple VALUES ({0}, {0})".format(i))
time.sleep(1)
@ -67,7 +68,8 @@ def test_chroot_with_different_root():
CREATE TABLE simple (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192);
'''.format(replica=node.name))
node.query("INSERT INTO simple VALUES ({0}, {0})".format(i))
for j in range(2): # Second insert to test deduplication
node.query("INSERT INTO simple VALUES ({0}, {0})".format(i))
assert node1.query('select count() from simple').strip() == '1'
assert node2.query('select count() from simple').strip() == '1'

View File

@ -0,0 +1,14 @@
1 6 3 3
1 6 3 3
1 6 [3,2]
1 6 [3,2]
1 0.5
1 0.5
1 0.1
1 0.1
0 333333 53
1 333333 53
2 333333 53
0 333333 53
1 333333 53
2 333333 53

View File

@ -0,0 +1,115 @@
drop table if exists test.summing_merge_tree_aggregate_function;
drop table if exists test.summing_merge_tree_null;
---- sum + uniq + uniqExact
create table test.summing_merge_tree_aggregate_function (
d materialized today(),
k UInt64,
c UInt64,
u AggregateFunction(uniq, UInt8),
ue AggregateFunction(uniqExact, UInt8)
) engine=SummingMergeTree(d, k, 8192);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(1), uniqExactState(1);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(2), uniqExactState(2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(3), uniqExactState(2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(1), uniqExactState(1);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(2), uniqExactState(2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(3), uniqExactState(3);
select
k, sum(c),
uniqMerge(u), uniqExactMerge(ue)
from test.summing_merge_tree_aggregate_function group by k;
optimize table test.summing_merge_tree_aggregate_function;
select
k, sum(c),
uniqMerge(u), uniqExactMerge(ue)
from test.summing_merge_tree_aggregate_function group by k;
drop table test.summing_merge_tree_aggregate_function;
---- sum + topK
create table test.summing_merge_tree_aggregate_function (d materialized today(), k UInt64, c UInt64, x AggregateFunction(topK(2), UInt8)) engine=SummingMergeTree(d, k, 8192);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(1);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(3);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(3);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(3);
select k, sum(c), topKMerge(2)(x) from test.summing_merge_tree_aggregate_function group by k;
optimize table test.summing_merge_tree_aggregate_function;
select k, sum(c), topKMerge(2)(x) from test.summing_merge_tree_aggregate_function group by k;
drop table test.summing_merge_tree_aggregate_function;
---- avg
create table test.summing_merge_tree_aggregate_function (d materialized today(), k UInt64, x AggregateFunction(avg, Float64)) engine=SummingMergeTree(d, k, 8192);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.0);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.1);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.2);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.3);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.4);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.5);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.6);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.7);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.8);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.9);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(1.0);
select k, avgMerge(x) from test.summing_merge_tree_aggregate_function group by k;
optimize table test.summing_merge_tree_aggregate_function;
select k, avgMerge(x) from test.summing_merge_tree_aggregate_function group by k;
drop table test.summing_merge_tree_aggregate_function;
---- quantile
create table test.summing_merge_tree_aggregate_function (d materialized today(), k UInt64, x AggregateFunction(quantile(0.1), Float64)) engine=SummingMergeTree(d, k, 8192);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.0);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.1);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.2);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.3);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.4);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.5);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.6);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.7);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.8);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.9);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(1.0);
select k, quantileMerge(0.1)(x) from test.summing_merge_tree_aggregate_function group by k;
optimize table test.summing_merge_tree_aggregate_function;
select k, quantileMerge(0.1)(x) from test.summing_merge_tree_aggregate_function group by k;
drop table test.summing_merge_tree_aggregate_function;
---- sum + uniq with more data
create table test.summing_merge_tree_null (
d materialized today(),
k UInt64,
c UInt64,
u UInt64
) engine=Null;
create materialized view test.summing_merge_tree_aggregate_function (
d materialized today(),
k UInt64,
c UInt64,
u AggregateFunction(uniq, UInt64)
) engine=SummingMergeTree(d, k, 8192)
as select d, k, sum(c) as c, uniqState(u) as u
from test.summing_merge_tree_null
group by d, k;
-- prime number 53 to avoid resonanse between %3 and %53
insert into test.summing_merge_tree_null select number % 3, 1, number % 53 from numbers(999999);
select k, sum(c), uniqMerge(u) from test.summing_merge_tree_aggregate_function group by k order by k;
optimize table test.summing_merge_tree_aggregate_function;
select k, sum(c), uniqMerge(u) from test.summing_merge_tree_aggregate_function group by k order by k;
drop table test.summing_merge_tree_aggregate_function;
drop table test.summing_merge_tree_null;

View File

@ -252,24 +252,24 @@ ghij
1
0
1
0
0
1
0
0
1
0
1
1
0
1
0
1
1
1
0
0
0
1
0
1
0
0
0
1
1
1
1
@ -284,13 +284,13 @@ ghij
0
1
1
0
0
0
1
0
0
0
1
1
1
1
1
1
1
----- Aggregation -----
A 0 2

View File

@ -14,9 +14,6 @@ SELECT * FROM test.merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.nu
SELECT * FROM test.merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id;
SELECT * FROM test.merge_tree_in_subqueries WHERE name IN (SELECT 'test' || toString(number) FROM system.numbers LIMIT 2, 3) ORDER BY id;
/* This should be removed when the issue of using the index for tuples in the IN operator is addressed. */
SET force_primary_key = 0;
SELECT id AS id2, name AS value FROM test.merge_tree_in_subqueries WHERE (value, id2) IN (SELECT 'test' || toString(number), number FROM system.numbers LIMIT 2, 3) ORDER BY id;
-- Non-index scans.

View File

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