Merge branch 'master' into join_regression_fix

This commit is contained in:
mergify[bot] 2022-06-15 11:53:42 +00:00 committed by GitHub
commit 2cb9579234
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
122 changed files with 2607 additions and 1745 deletions

View File

@ -4,7 +4,7 @@
**[ClickHouse release v22.3-lts, 2022-03-17](#223)**<br>
**[ClickHouse release v22.2, 2022-02-17](#222)**<br>
**[ClickHouse release v22.1, 2022-01-18](#221)**<br>
**[Changelog for 2021](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/whats-new/changelog/2021.md)**<br>
**[Changelog for 2021](https://clickhouse.com/docs/en/whats-new/changelog/2021/)**<br>
### <a id="225"></a> ClickHouse release 22.5, 2022-05-19
@ -172,7 +172,7 @@
#### Backward Incompatible Change
* Do not allow SETTINGS after FORMAT for INSERT queries (there is compatibility setting `parser_settings_after_format_compact` to accept such queries, but it is turned OFF by default). [#35883](https://github.com/ClickHouse/ClickHouse/pull/35883) ([Azat Khuzhin](https://github.com/azat)).
* Do not allow SETTINGS after FORMAT for INSERT queries (there is compatibility setting `allow_settings_after_format_in_insert` to accept such queries, but it is turned OFF by default). [#35883](https://github.com/ClickHouse/ClickHouse/pull/35883) ([Azat Khuzhin](https://github.com/azat)).
* Function `yandexConsistentHash` (consistent hashing algorithm by Konstantin "kostik" Oblakov) is renamed to `kostikConsistentHash`. The old name is left as an alias for compatibility. Although this change is backward compatible, we may remove the alias in subsequent releases, that's why it's recommended to update the usages of this function in your apps. [#35553](https://github.com/ClickHouse/ClickHouse/pull/35553) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### New Feature

View File

@ -13,7 +13,3 @@ ClickHouse® is an open-source column-oriented database management system that a
* [Code Browser (Woboq)](https://clickhouse.com/codebrowser/ClickHouse/index.html) with syntax highlight and navigation.
* [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlight, powered by github.dev.
* [Contacts](https://clickhouse.com/company/#contact) can help to get your questions answered if there are any.
## Upcoming Events
* [ClickHouse Meetup Amsterdam (in-person and online)](https://www.meetup.com/clickhouse-netherlands-user-group/events/286017044/) on June 8th, 2022

View File

@ -6,10 +6,10 @@ sidebar_position: 33
Calculates the amount `Σ((x - x̅)^2) / (n - 1)`, where `n` is the sample size and `x̅`is the average value of `x`.
It represents an unbiased estimate of the variance of a random variable if passed values form its sample.
It represents an unbiased estimate of the variance of a random variable if passed values from its sample.
Returns `Float64`. When `n <= 1`, returns `+∞`.
:::note
This function uses a numerically unstable algorithm. If you need [numerical stability](https://en.wikipedia.org/wiki/Numerical_stability) in calculations, use the `varSampStable` function. It works slower but provides a lower computational error.
:::
:::

View File

@ -252,12 +252,14 @@ This is an experimental feature that may change in backwards-incompatible ways i
:::
``` sql
CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [ENGINE = engine] [WATERMARK = strategy] [ALLOWED_LATENESS = interval_function] AS SELECT ... GROUP BY time_window_function
CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [INNER ENGINE engine] [ENGINE engine] [WATERMARK strategy] [ALLOWED_LATENESS interval_function] [POPULATE] AS SELECT ... GROUP BY time_window_function
```
Window view can aggregate data by time window and output the results when the window is ready to fire. It stores the partial aggregation results in an inner(or specified) table to reduce latency and can push the processing result to a specified table or push notifications using the WATCH query.
Creating a window view is similar to creating `MATERIALIZED VIEW`. Window view needs an inner storage engine to store intermediate data. The inner storage will use `AggregatingMergeTree` as the default engine.
Creating a window view is similar to creating `MATERIALIZED VIEW`. Window view needs an inner storage engine to store intermediate data. The inner storage can be specified by using `INNER ENGINE` clause, the window view will use `AggregatingMergeTree` as the default inner engine.
When creating a window view without `TO [db].[table]`, you must specify `ENGINE` the table engine for storing data.
### Time Window Functions
@ -297,6 +299,8 @@ CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTE
Note that elements emitted by a late firing should be treated as updated results of a previous computation. Instead of firing at the end of windows, the window view will fire immediately when the late event arrives. Thus, it will result in multiple outputs for the same window. Users need to take these duplicated results into account or deduplicate them.
You can modify `SELECT` query that was specified in the window view by using `ALTER TABLE … MODIFY QUERY` statement. The data structure resulting in a new `SELECT` query should be the same as the original `SELECT` query when with or without `TO [db.]name` clause. Note that the data in the current window will be lost because the intermediate state cannot be reused.
### Monitoring New Windows
Window view supports the [WATCH](../../../sql-reference/statements/watch.md) query to monitoring changes, or use `TO` syntax to output the results to a table.
@ -314,6 +318,7 @@ WATCH [db.]window_view
- `window_view_clean_interval`: The clean interval of window view in seconds to free outdated data. The system will retain the windows that have not been fully triggered according to the system time or `WATERMARK` configuration, and the other data will be deleted.
- `window_view_heartbeat_interval`: The heartbeat interval in seconds to indicate the watch query is alive.
- `wait_for_window_view_fire_signal_timeout`: Timeout for waiting for window view fire signal in event time processing.
### Example

View File

@ -32,6 +32,7 @@ The list of available `SYSTEM` statements:
- [START TTL MERGES](#query_language-start-ttl-merges)
- [STOP MOVES](#query_language-stop-moves)
- [START MOVES](#query_language-start-moves)
- [SYSTEM UNFREEZE](#query_language-system-unfreeze)
- [STOP FETCHES](#query_language-system-stop-fetches)
- [START FETCHES](#query_language-system-start-fetches)
- [STOP REPLICATED SENDS](#query_language-system-start-replicated-sends)
@ -239,6 +240,14 @@ Returns `Ok.` even if table does not exist. Returns error when database does not
SYSTEM START MOVES [[db.]merge_tree_family_table_name]
```
### SYSTEM UNFREEZE {#query_language-system-unfreeze}
Clears freezed backup with the specified name from all the disks. See more about unfreezing separate parts in [ALTER TABLE table_name UNFREEZE WITH NAME ](alter/partition.md#alter_unfreeze-partition)
``` sql
SYSTEM UNFREEZE WITH NAME <backup_name>
```
## Managing ReplicatedMergeTree Tables
ClickHouse can manage background replication related processes in [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md#table_engines-replication) tables.

View File

@ -22,9 +22,9 @@ ClickHouse позволяет автоматически удалять данн
ClickHouse не удаляет данные в реальном времени, как СУБД [OLTP](https://en.wikipedia.org/wiki/Online_transaction_processing). Больше всего на такое удаление похожи мутации. Они выполняются с помощью запросов `ALTER ... DELETE` или `ALTER ... UPDATE`. В отличие от обычных запросов `DELETE` и `UPDATE`, мутации выполняются асинхронно, в пакетном режиме, не в реальном времени. В остальном после слов `ALTER TABLE` синтаксис обычных запросов и мутаций одинаковый.
`ALTER DELETE` можно использовать для гибкого удаления устаревших данных. Если вам нужно делать это регулярно, единственный недостаток такого подхода будет заключаться в том, что потребуется внешняя система для запуска запроса. Кроме того, могут возникнуть некоторые проблемы с производительностью, поскольку мутации перезаписывают целые куски данных если в них содержится хотя бы одна строка, которую нужно удалить.
`ALTER DELETE` можно использовать для гибкого удаления устаревших данных. Если вам нужно делать это регулярно, основной недостаток такого подхода будет заключаться в том, что потребуется внешняя система для запуска запроса. Кроме того, могут возникнуть некоторые проблемы с производительностью, поскольку мутации перезаписывают целые куски данных если в них содержится хотя бы одна строка, которую нужно удалить.
Это самый распространенный подход к тому, чтобы обеспечить соблюдение принципов [GDPR](https://gdpr-info.eu) в вашей системе на ClickHouse.
Это - самый распространенный подход к тому, чтобы обеспечить соблюдение принципов [GDPR](https://gdpr-info.eu) в вашей системе на ClickHouse.
Подробнее смотрите в разделе [Мутации](../../sql-reference/statements/alter/index.md#alter-mutations).

View File

@ -30,6 +30,7 @@ sidebar_label: SYSTEM
- [START TTL MERGES](#query_language-start-ttl-merges)
- [STOP MOVES](#query_language-stop-moves)
- [START MOVES](#query_language-start-moves)
- [SYSTEM UNFREEZE](#query_language-system-unfreeze)
- [STOP FETCHES](#query_language-system-stop-fetches)
- [START FETCHES](#query_language-system-start-fetches)
- [STOP REPLICATED SENDS](#query_language-system-start-replicated-sends)
@ -235,6 +236,14 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
SYSTEM START MOVES [[db.]merge_tree_family_table_name]
```
### SYSTEM UNFREEZE {#query_language-system-unfreeze}
Удаляет с диска все "замороженные" партиции данного бэкапа. Про удаление партиций по отдельности смотрите запрос [ALTER TABLE table_name UNFREEZE WITH NAME ](alter/partition.md#alter_unfreeze-partition)
``` sql
SYSTEM UNFREEZE WITH NAME <backup_name>
```
## Managing ReplicatedMergeTree Tables {#query-language-system-replicated}
ClickHouse может управлять фоновыми процессами связанными c репликацией в таблицах семейства [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md).

View File

@ -5,4 +5,4 @@ sidebar_position: 82
# Что нового в ClickHouse?
Планы развития вкратце изложены [здесь](https://github.com/ClickHouse/ClickHouse/issues/17623), а новости по предыдущим релизам подробно описаны в [журнале изменений](./changelog/).
Планы развития вкратце изложены [здесь](https://github.com/ClickHouse/ClickHouse/issues/32513), а новости по предыдущим релизам подробно описаны в [журнале изменений](./changelog/).

View File

@ -250,12 +250,14 @@ Code: 60. DB::Exception: Received from localhost:9000. DB::Exception: Table defa
`set allow_experimental_window_view = 1`
``` sql
CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [ENGINE = engine] [WATERMARK = strategy] [ALLOWED_LATENESS = interval_function] AS SELECT ... GROUP BY time_window_function
CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [INNER ENGINE engine] [ENGINE engine] [WATERMARK strategy] [ALLOWED_LATENESS interval_function] [POPULATE] AS SELECT ... GROUP BY time_window_function
```
Window view可以通过时间窗口聚合数据并在满足窗口触发条件时自动触发对应窗口计算。其通过将计算状态保存降低处理延迟支持将处理结果输出至目标表或通过`WATCH`语句输出至终端。
创建window view的方式和创建物化视图类似。Window view使用默认为`AggregatingMergeTree`的内部存储引擎存储计算中间状态。
创建window view的方式和创建物化视图类似。Window view通过`INNER ENGINE`指定内部存储引擎以存储窗口计算中间状态,默认使用`AggregatingMergeTree`作为内部中间状态存储引擎。
创建不带`TO [db].[table]`的window view时必须指定`ENGINE` 用于存储数据的表引擎。
### 时间窗口函数 {#window-view-shi-jian-chuang-kou-han-shu}
@ -295,6 +297,10 @@ CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTE
需要注意的是迟到消息需要更新之前的处理结果。与在窗口结束时触发不同迟到消息到达时window view会立即触发计算。因此会导致同一个窗口输出多次计算结果。用户需要注意这种情况并消除重复结果。
### 查询语句修改 {#window-view-cha-xun-yu-ju-xiu-gai}
用户可以通过`ALTER TABLE ... MODIFY QUERY`语句修改window view的`SELECT`查询语句。无论是否使用`TO [db.]name`语句,新`SELECT`语句的数据结构均需和旧语句相同。需要注意的是,由于窗口计算中间状态无法复用,修改查询语句时会丢失当前窗口数据。
### 新窗口监控 {#window-view-xin-chuang-kou-jian-kong}
Window view可以通过`WATCH`语句将处理结果推送至终端,或通过`TO`语句将结果推送至数据表。
@ -309,6 +315,7 @@ WATCH [db.]name [LIMIT n]
- `window_view_clean_interval`: window view清除过期数据间隔(单位为秒)。系统会定期清除过期数据,尚未触发的窗口数据不会被清除。
- `window_view_heartbeat_interval`: 用于判断watch查询活跃的心跳时间间隔。
- `wait_for_window_view_fire_signal_timeout`: Event time 处理模式下,窗口触发信号等待超时时间。
### 示例 {#window-view-shi-li}

View File

@ -26,6 +26,7 @@ sidebar_label: SYSTEM
- [START TTL MERGES](#query_language-start-ttl-merges)
- [STOP MOVES](#query_language-stop-moves)
- [START MOVES](#query_language-start-moves)
- [SYSTEM UNFREEZE](#query_language-system-unfreeze)
- [STOP FETCHES](#query_language-system-stop-fetches)
- [START FETCHES](#query_language-system-start-fetches)
- [STOP REPLICATED SENDS](#query_language-system-start-replicated-sends)
@ -203,6 +204,14 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
```
### SYSTEM UNFREEZE {#query_language-system-unfreeze}
从所有磁盘中清除具有指定名称的冻结备份。 查看更多关于解冻单独部分的信息 [ALTER TABLE table_name UNFREEZE WITH NAME ](alter/partition.md#alter_unfreeze-partition)
``` sql
SYSTEM UNFREEZE WITH NAME <backup_name>
```
## Managing ReplicatedMergeTree Tables {#query-language-system-replicated}
管理 [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md)表的后台复制相关进程。

View File

@ -164,6 +164,7 @@ enum class AccessType
M(SYSTEM_FLUSH_LOGS, "FLUSH LOGS", GLOBAL, SYSTEM_FLUSH) \
M(SYSTEM_FLUSH, "", GROUP, SYSTEM) \
M(SYSTEM_THREAD_FUZZER, "SYSTEM START THREAD FUZZER, SYSTEM STOP THREAD FUZZER, START THREAD FUZZER, STOP THREAD FUZZER", GLOBAL, SYSTEM) \
M(SYSTEM_UNFREEZE, "SYSTEM UNFREEZE", GLOBAL, SYSTEM) \
M(SYSTEM, "", GROUP, ALL) /* allows to execute SYSTEM {SHUTDOWN|RELOAD CONFIG|...} */ \
\
M(dictGet, "dictHas, dictGetHierarchy, dictIsIn", DICTIONARY, ALL) /* allows to execute functions dictGet(), dictHas(), dictGetHierarchy(), dictIsIn() */\

View File

@ -326,7 +326,7 @@ Strings BackupCoordinationDistributed::listFiles(const String & prefix, const St
elements.push_back(String{new_element});
}
std::sort(elements.begin(), elements.end());
::sort(elements.begin(), elements.end());
return elements;
}

View File

@ -84,7 +84,7 @@ namespace
return true;
});
std::sort(res.begin(), res.end());
::sort(res.begin(), res.end());
res.erase(std::unique(res.begin(), res.end()), res.end());
return res;
}
@ -113,7 +113,7 @@ namespace
return true;
});
std::sort(res.begin(), res.end());
::sort(res.begin(), res.end());
res.erase(std::unique(res.begin(), res.end()), res.end());
return res;
}

View File

@ -950,18 +950,17 @@ void ClientBase::onProfileEvents(Block & block)
progress_indication.addThreadIdToList(host_name, thread_id);
auto event_name = names.getDataAt(i);
auto value = array_values[i];
/// Ignore negative time delta or memory usage just in case.
if (value < 0)
continue;
if (event_name == user_time_name)
{
thread_times[host_name][thread_id].user_ms = value;
}
else if (event_name == system_time_name)
{
thread_times[host_name][thread_id].system_ms = value;
}
else if (event_name == MemoryTracker::USAGE_EVENT_NAME)
{
thread_times[host_name][thread_id].memory_usage = value;
}
}
auto elapsed_time = profile_events.watch.elapsedMicroseconds();
progress_indication.updateThreadEventData(thread_times, elapsed_time);

View File

@ -318,7 +318,59 @@ template <typename T>
void ColumnVector<T>::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const
{
auto sort = [](auto begin, auto end, auto pred) { ::sort(begin, end, pred); };
bool reverse = direction == IColumn::PermutationSortDirection::Descending;
bool ascending = direction == IColumn::PermutationSortDirection::Ascending;
bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable;
auto sort = [&](auto begin, auto end, auto pred)
{
/// A case for radix sort
if constexpr (is_arithmetic_v<T> && !is_big_int_v<T>)
{
/// TODO: LSD RadixSort is currently not stable if direction is descending, or value is floating point
bool use_radix_sort = (sort_is_stable && ascending && !std::is_floating_point_v<T>) || !sort_is_stable;
size_t size = end - begin;
/// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters.
if (size >= 256 && size <= std::numeric_limits<UInt32>::max() && use_radix_sort)
{
PaddedPODArray<ValueWithIndex<T>> pairs(size);
size_t index = 0;
for (auto it = begin; it != end; ++it)
{
pairs[index] = {data[*it], static_cast<UInt32>(*it)};
++index;
}
RadixSort<RadixSortTraits<T>>::executeLSD(pairs.data(), size, reverse, begin);
/// Radix sort treats all NaNs to be greater than all numbers.
/// If the user needs the opposite, we must move them accordingly.
if (std::is_floating_point_v<T> && nan_direction_hint < 0)
{
size_t nans_to_move = 0;
for (size_t i = 0; i < size; ++i)
{
if (isNaN(data[begin[reverse ? i : size - 1 - i]]))
++nans_to_move;
else
break;
}
if (nans_to_move)
{
std::rotate(begin, begin + (reverse ? nans_to_move : size - nans_to_move), end);
}
}
return;
}
}
::sort(begin, end, pred);
};
auto partial_sort = [](auto begin, auto mid, auto end, auto pred) { ::partial_sort(begin, mid, end, pred); };
if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable)

View File

@ -90,7 +90,7 @@ public:
/// Creates column with the same type and specified size.
/// If size is less current size, then data is cut.
/// If size is greater, than default values are appended.
[[nodiscard]] virtual MutablePtr cloneResized(size_t /*size*/) const { throw Exception("Cannot cloneResized() column " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
[[nodiscard]] virtual MutablePtr cloneResized(size_t /*size*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cloneResized() column {}", getName()); }
/// Returns number of values in column.
[[nodiscard]] virtual size_t size() const = 0;

View File

@ -19,7 +19,7 @@ namespace
double calculateCPUUsage(DB::ThreadIdToTimeMap times, UInt64 elapsed)
{
auto accumulated = std::accumulate(times.begin(), times.end(), 0,
[](Int64 acc, const auto & elem)
[](UInt64 acc, const auto & elem)
{
if (elem.first == ALL_THREADS)
return acc;
@ -191,6 +191,10 @@ void ProgressIndication::writeProgress()
{
WriteBufferFromOwnString profiling_msg_builder;
/// We don't want -0. that can appear due to rounding errors.
if (cpu_usage <= 0)
cpu_usage = 0;
profiling_msg_builder << "(" << fmt::format("{:.1f}", cpu_usage) << " CPU";
if (memory_usage > 0)

View File

@ -16,11 +16,11 @@ namespace DB
struct ThreadEventData
{
Int64 time() const noexcept { return user_ms + system_ms; }
UInt64 time() const noexcept { return user_ms + system_ms; }
Int64 user_ms = 0;
Int64 system_ms = 0;
Int64 memory_usage = 0;
UInt64 user_ms = 0;
UInt64 system_ms = 0;
UInt64 memory_usage = 0;
};
using ThreadIdToTimeMap = std::unordered_map<UInt64, ThreadEventData>;

View File

@ -7,6 +7,7 @@
#include <filesystem>
#include <base/find_symbols.h>
#include <base/sort.h>
#include <base/getFQDNOrHostName.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Exception.h>
@ -169,7 +170,7 @@ std::vector<ShuffleHost> ZooKeeper::shuffleHosts() const
shuffle_hosts.emplace_back(shuffle_host);
}
std::sort(
::sort(
shuffle_hosts.begin(), shuffle_hosts.end(),
[](const ShuffleHost & lhs, const ShuffleHost & rhs)
{

View File

@ -280,7 +280,15 @@ time_t getModificationTime(const std::string & path)
struct stat st;
if (stat(path.c_str(), &st) == 0)
return st.st_mtime;
DB::throwFromErrnoWithPath("Cannot check modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
DB::throwFromErrnoWithPath("Cannot check modification time for file: " + path, path, DB::ErrorCodes::CANNOT_STAT);
}
time_t getChangeTime(const std::string & path)
{
struct stat st;
if (stat(path.c_str(), &st) == 0)
return st.st_ctime;
DB::throwFromErrnoWithPath("Cannot check change time for file: " + path, path, DB::ErrorCodes::CANNOT_STAT);
}
Poco::Timestamp getModificationTimestamp(const std::string & path)

View File

@ -75,7 +75,10 @@ bool canRead(const std::string & path);
bool canWrite(const std::string & path);
bool canExecute(const std::string & path);
/// st_mtime
time_t getModificationTime(const std::string & path);
Poco::Timestamp getModificationTimestamp(const std::string & path);
void setModificationTime(const std::string & path, time_t time);
/// st_ctime
time_t getChangeTime(const std::string & path);
}

View File

@ -0,0 +1,511 @@
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionInfo.h>
#include <Compression/CompressionFactory.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <span>
#include <bit>
#include <concepts>
namespace DB
{
class CompressionCodecFPC : public ICompressionCodec
{
public:
CompressionCodecFPC(UInt8 float_size, UInt8 compression_level);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
static constexpr UInt8 MAX_COMPRESSION_LEVEL{28};
static constexpr UInt8 DEFAULT_COMPRESSION_LEVEL{12};
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
private:
static constexpr UInt32 HEADER_SIZE{3};
UInt8 float_width; // size of uncompressed float in bytes
UInt8 level; // compression level, 2^level * float_width is the size of predictors table in bytes
};
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;
extern const int CANNOT_DECOMPRESS;
extern const int ILLEGAL_CODEC_PARAMETER;
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int BAD_ARGUMENTS;
}
uint8_t CompressionCodecFPC::getMethodByte() const
{
return static_cast<uint8_t>(CompressionMethodByte::FPC);
}
void CompressionCodecFPC::updateHash(SipHash & hash) const
{
getCodecDesc()->updateTreeHash(hash);
}
CompressionCodecFPC::CompressionCodecFPC(UInt8 float_size, UInt8 compression_level)
: float_width{float_size}, level{compression_level}
{
setCodecDescription("FPC", {std::make_shared<ASTLiteral>(static_cast<UInt64>(level))});
}
UInt32 CompressionCodecFPC::getMaxCompressedDataSize(UInt32 uncompressed_size) const
{
auto float_count = (uncompressed_size + float_width - 1) / float_width;
if (float_count % 2 != 0)
++float_count;
return HEADER_SIZE + float_count * float_width + float_count / 2;
}
namespace
{
UInt8 getFloatBytesSize(const IDataType & column_type)
{
if (!WhichDataType(column_type).isFloat())
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "FPC codec is not applicable for {} because the data type is not float",
column_type.getName());
}
if (auto float_size = column_type.getSizeOfValueInMemory(); float_size >= 4)
{
return static_cast<UInt8>(float_size);
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "FPC codec is not applicable for floats of size less than 4 bytes. Given type {}",
column_type.getName());
}
std::byte encodeEndianness(std::endian endian)
{
switch (endian)
{
case std::endian::little:
return std::byte{0};
case std::endian::big:
return std::byte{1};
}
throw Exception("Unsupported endianness", ErrorCodes::BAD_ARGUMENTS);
}
std::endian decodeEndianness(std::byte endian)
{
switch (std::to_integer<unsigned char>(endian))
{
case 0:
return std::endian::little;
case 1:
return std::endian::big;
}
throw Exception("Unsupported endianness", ErrorCodes::BAD_ARGUMENTS);
}
}
void registerCodecFPC(CompressionCodecFactory & factory)
{
auto method_code = static_cast<UInt8>(CompressionMethodByte::FPC);
auto codec_builder = [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr
{
UInt8 float_width{0};
if (column_type != nullptr)
float_width = getFloatBytesSize(*column_type);
UInt8 level = CompressionCodecFPC::DEFAULT_COMPRESSION_LEVEL;
if (arguments && !arguments->children.empty())
{
if (arguments->children.size() > 1)
{
throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE,
"FPC codec must have 1 parameter, given {}", arguments->children.size());
}
const auto * literal = arguments->children.front()->as<ASTLiteral>();
if (!literal)
throw Exception("FPC codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER);
level = literal->value.safeGet<UInt8>();
if (level == 0)
throw Exception("FPC codec level must be at least 1", ErrorCodes::ILLEGAL_CODEC_PARAMETER);
if (level > CompressionCodecFPC::MAX_COMPRESSION_LEVEL)
throw Exception("FPC codec level must be at most 28", ErrorCodes::ILLEGAL_CODEC_PARAMETER);
}
return std::make_shared<CompressionCodecFPC>(float_width, level);
};
factory.registerCompressionCodecWithType("FPC", method_code, codec_builder);
}
namespace
{
template <std::unsigned_integral TUint>
requires (sizeof(TUint) >= 4)
class DfcmPredictor
{
public:
explicit DfcmPredictor(std::size_t table_size): table(table_size, 0), prev_value{0}, hash{0}
{
}
[[nodiscard]]
TUint predict() const noexcept
{
return table[hash] + prev_value;
}
void add(TUint value) noexcept
{
table[hash] = value - prev_value;
recalculateHash();
prev_value = value;
}
private:
void recalculateHash() noexcept
{
auto value = table[hash];
if constexpr (sizeof(TUint) >= 8)
{
hash = ((hash << 2) ^ static_cast<std::size_t>(value >> 40)) & (table.size() - 1);
}
else
{
hash = ((hash << 4) ^ static_cast<std::size_t>(value >> 23)) & (table.size() - 1);
}
}
std::vector<TUint> table;
TUint prev_value;
std::size_t hash;
};
template <std::unsigned_integral TUint>
requires (sizeof(TUint) >= 4)
class FcmPredictor
{
public:
explicit FcmPredictor(std::size_t table_size): table(table_size, 0), hash{0}
{
}
[[nodiscard]]
TUint predict() const noexcept
{
return table[hash];
}
void add(TUint value) noexcept
{
table[hash] = value;
recalculateHash();
}
private:
void recalculateHash() noexcept
{
auto value = table[hash];
if constexpr (sizeof(TUint) >= 8)
{
hash = ((hash << 6) ^ static_cast<std::size_t>(value >> 48)) & (table.size() - 1);
}
else
{
hash = ((hash << 1) ^ static_cast<std::size_t>(value >> 22)) & (table.size() - 1);
}
}
std::vector<TUint> table;
std::size_t hash;
};
template <std::unsigned_integral TUint, std::endian Endian = std::endian::native>
requires (Endian == std::endian::little || Endian == std::endian::big)
class FPCOperation
{
static constexpr std::size_t CHUNK_SIZE{64};
static constexpr auto VALUE_SIZE = sizeof(TUint);
static constexpr std::byte FCM_BIT{0};
static constexpr std::byte DFCM_BIT{1u << 3};
static constexpr auto DFCM_BIT_1 = DFCM_BIT << 4;
static constexpr auto DFCM_BIT_2 = DFCM_BIT;
static constexpr unsigned MAX_ZERO_BYTE_COUNT{0b111u};
public:
FPCOperation(std::span<std::byte> destination, UInt8 compression_level)
: dfcm_predictor(1u << compression_level), fcm_predictor(1u << compression_level), chunk{}, result{destination}
{
}
std::size_t encode(std::span<const std::byte> data) &&
{
auto initial_size = result.size();
std::span chunk_view(chunk);
for (std::size_t i = 0; i < data.size(); i += chunk_view.size_bytes())
{
auto written_values = importChunk(data.subspan(i), chunk_view);
encodeChunk(chunk_view.subspan(0, written_values));
}
return initial_size - result.size();
}
void decode(std::span<const std::byte> values, std::size_t decoded_size) &&
{
std::size_t read_bytes{0};
std::span<TUint> chunk_view(chunk);
for (std::size_t i = 0; i < decoded_size; i += chunk_view.size_bytes())
{
if (i + chunk_view.size_bytes() > decoded_size)
chunk_view = chunk_view.first(ceilBytesToEvenValues(decoded_size - i));
read_bytes += decodeChunk(values.subspan(read_bytes), chunk_view);
exportChunk(chunk_view);
}
}
private:
static std::size_t ceilBytesToEvenValues(std::size_t bytes_count)
{
auto values_count = (bytes_count + VALUE_SIZE - 1) / VALUE_SIZE;
return values_count % 2 == 0 ? values_count : values_count + 1;
}
std::size_t importChunk(std::span<const std::byte> values, std::span<TUint> chnk)
{
if (auto chunk_view = std::as_writable_bytes(chnk); chunk_view.size() <= values.size())
{
std::memcpy(chunk_view.data(), values.data(), chunk_view.size());
return chunk_view.size() / VALUE_SIZE;
}
else
{
std::memset(chunk_view.data(), 0, chunk_view.size());
std::memcpy(chunk_view.data(), values.data(), values.size());
return ceilBytesToEvenValues(values.size());
}
}
void exportChunk(std::span<const TUint> chnk)
{
auto chunk_view = std::as_bytes(chnk).first(std::min(result.size(), chnk.size_bytes()));
std::memcpy(result.data(), chunk_view.data(), chunk_view.size());
result = result.subspan(chunk_view.size());
}
void encodeChunk(std::span<const TUint> seq)
{
for (std::size_t i = 0; i < seq.size(); i += 2)
{
encodePair(seq[i], seq[i + 1]);
}
}
struct CompressedValue
{
TUint value;
unsigned compressed_size;
std::byte predictor;
};
unsigned encodeCompressedZeroByteCount(int compressed)
{
if constexpr (VALUE_SIZE == MAX_ZERO_BYTE_COUNT + 1)
{
if (compressed >= 4)
--compressed;
}
return std::min(static_cast<unsigned>(compressed), MAX_ZERO_BYTE_COUNT);
}
unsigned decodeCompressedZeroByteCount(unsigned encoded_size)
{
if constexpr (VALUE_SIZE == MAX_ZERO_BYTE_COUNT + 1)
{
if (encoded_size > 3)
++encoded_size;
}
return encoded_size;
}
CompressedValue compressValue(TUint value) noexcept
{
static constexpr auto BITS_PER_BYTE = std::numeric_limits<unsigned char>::digits;
TUint compressed_dfcm = dfcm_predictor.predict() ^ value;
TUint compressed_fcm = fcm_predictor.predict() ^ value;
dfcm_predictor.add(value);
fcm_predictor.add(value);
auto zeroes_dfcm = std::countl_zero(compressed_dfcm);
auto zeroes_fcm = std::countl_zero(compressed_fcm);
if (zeroes_dfcm > zeroes_fcm)
return {compressed_dfcm, encodeCompressedZeroByteCount(zeroes_dfcm / BITS_PER_BYTE), DFCM_BIT};
return {compressed_fcm, encodeCompressedZeroByteCount(zeroes_fcm / BITS_PER_BYTE), FCM_BIT};
}
void encodePair(TUint first, TUint second)
{
auto [value1, zero_byte_count1, predictor1] = compressValue(first);
auto [value2, zero_byte_count2, predictor2] = compressValue(second);
std::byte header{0x0};
header |= (predictor1 << 4) | predictor2;
header |= static_cast<std::byte>((zero_byte_count1 << 4) | zero_byte_count2);
result.front() = header;
zero_byte_count1 = decodeCompressedZeroByteCount(zero_byte_count1);
zero_byte_count2 = decodeCompressedZeroByteCount(zero_byte_count2);
auto tail_size1 = VALUE_SIZE - zero_byte_count1;
auto tail_size2 = VALUE_SIZE - zero_byte_count2;
std::memcpy(result.data() + 1, valueTail(value1, zero_byte_count1), tail_size1);
std::memcpy(result.data() + 1 + tail_size1, valueTail(value2, zero_byte_count2), tail_size2);
result = result.subspan(1 + tail_size1 + tail_size2);
}
std::size_t decodeChunk(std::span<const std::byte> values, std::span<TUint> seq)
{
std::size_t read_bytes{0};
for (std::size_t i = 0; i < seq.size(); i += 2)
{
read_bytes += decodePair(values.subspan(read_bytes), seq[i], seq[i + 1]);
}
return read_bytes;
}
TUint decompressValue(TUint value, bool isDfcmPredictor)
{
TUint decompressed;
if (isDfcmPredictor)
{
decompressed = dfcm_predictor.predict() ^ value;
}
else
{
decompressed = fcm_predictor.predict() ^ value;
}
dfcm_predictor.add(decompressed);
fcm_predictor.add(decompressed);
return decompressed;
}
std::size_t decodePair(std::span<const std::byte> bytes, TUint& first, TUint& second)
{
if (bytes.empty())
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Unexpected end of encoded sequence");
auto zero_byte_count1 = decodeCompressedZeroByteCount(
std::to_integer<unsigned>(bytes.front() >> 4) & MAX_ZERO_BYTE_COUNT);
auto zero_byte_count2 = decodeCompressedZeroByteCount(
std::to_integer<unsigned>(bytes.front()) & MAX_ZERO_BYTE_COUNT);
auto tail_size1 = VALUE_SIZE - zero_byte_count1;
auto tail_size2 = VALUE_SIZE - zero_byte_count2;
if (bytes.size() < 1 + tail_size1 + tail_size2)
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Unexpected end of encoded sequence");
TUint value1{0};
TUint value2{0};
std::memcpy(valueTail(value1, zero_byte_count1), bytes.data() + 1, tail_size1);
std::memcpy(valueTail(value2, zero_byte_count2), bytes.data() + 1 + tail_size1, tail_size2);
auto is_dfcm_predictor1 = std::to_integer<unsigned char>(bytes.front() & DFCM_BIT_1) != 0;
auto is_dfcm_predictor2 = std::to_integer<unsigned char>(bytes.front() & DFCM_BIT_2) != 0;
first = decompressValue(value1, is_dfcm_predictor1);
second = decompressValue(value2, is_dfcm_predictor2);
return 1 + tail_size1 + tail_size2;
}
static void* valueTail(TUint& value, unsigned compressed_size)
{
if constexpr (Endian == std::endian::little)
{
return &value;
}
else
{
return reinterpret_cast<std::byte*>(&value) + compressed_size;
}
}
DfcmPredictor<TUint> dfcm_predictor;
FcmPredictor<TUint> fcm_predictor;
std::array<TUint, CHUNK_SIZE> chunk{};
std::span<std::byte> result{};
};
}
UInt32 CompressionCodecFPC::doCompressData(const char * source, UInt32 source_size, char * dest) const
{
dest[0] = static_cast<char>(float_width);
dest[1] = static_cast<char>(level);
dest[2] = std::to_integer<char>(encodeEndianness(std::endian::native));
auto dest_size = getMaxCompressedDataSize(source_size);
auto destination = std::as_writable_bytes(std::span(dest, dest_size).subspan(HEADER_SIZE));
auto src = std::as_bytes(std::span(source, source_size));
switch (float_width)
{
case sizeof(Float64):
return HEADER_SIZE + FPCOperation<UInt64>(destination, level).encode(src);
case sizeof(Float32):
return HEADER_SIZE + FPCOperation<UInt32>(destination, level).encode(src);
default:
break;
}
throw Exception("Cannot compress. File has incorrect float width", ErrorCodes::CANNOT_COMPRESS);
}
void CompressionCodecFPC::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const
{
if (source_size < HEADER_SIZE)
throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS);
auto compressed_data = std::as_bytes(std::span(source, source_size));
auto compressed_float_width = std::to_integer<UInt8>(compressed_data[0]);
auto compressed_level = std::to_integer<UInt8>(compressed_data[1]);
if (compressed_level == 0 || compressed_level > MAX_COMPRESSION_LEVEL)
throw Exception("Cannot decompress. File has incorrect level", ErrorCodes::CANNOT_DECOMPRESS);
if (decodeEndianness(compressed_data[2]) != std::endian::native)
throw Exception("Cannot decompress. File has incorrect endianness", ErrorCodes::CANNOT_DECOMPRESS);
auto destination = std::as_writable_bytes(std::span(dest, uncompressed_size));
auto src = compressed_data.subspan(HEADER_SIZE);
switch (compressed_float_width)
{
case sizeof(Float64):
FPCOperation<UInt64>(destination, compressed_level).decode(src, uncompressed_size);
break;
case sizeof(Float32):
FPCOperation<UInt32>(destination, compressed_level).decode(src, uncompressed_size);
break;
default:
throw Exception("Cannot decompress. File has incorrect float width", ErrorCodes::CANNOT_DECOMPRESS);
}
}
}

View File

@ -177,6 +177,7 @@ void registerCodecT64(CompressionCodecFactory & factory);
void registerCodecDoubleDelta(CompressionCodecFactory & factory);
void registerCodecGorilla(CompressionCodecFactory & factory);
void registerCodecEncrypted(CompressionCodecFactory & factory);
void registerCodecFPC(CompressionCodecFactory & factory);
#endif
@ -194,6 +195,7 @@ CompressionCodecFactory::CompressionCodecFactory()
registerCodecDoubleDelta(*this);
registerCodecGorilla(*this);
registerCodecEncrypted(*this);
registerCodecFPC(*this);
#endif
default_codec = get("LZ4", {});

View File

@ -44,7 +44,8 @@ enum class CompressionMethodByte : uint8_t
DoubleDelta = 0x94,
Gorilla = 0x95,
AES_128_GCM_SIV = 0x96,
AES_256_GCM_SIV = 0x97
AES_256_GCM_SIV = 0x97,
FPC = 0x98
};
}

View File

@ -36,21 +36,20 @@ std::string formatChangelogPath(const std::string & prefix, const ChangelogFileD
return path;
}
ChangelogFileDescription getChangelogFileDescription(const std::string & path_str)
ChangelogFileDescription getChangelogFileDescription(const std::filesystem::path & path)
{
std::filesystem::path path(path_str);
std::string filename = path.stem();
Strings filename_parts;
boost::split(filename_parts, filename, boost::is_any_of("_"));
if (filename_parts.size() < 3)
throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path_str);
throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path.generic_string());
ChangelogFileDescription result;
result.prefix = filename_parts[0];
result.from_log_index = parse<uint64_t>(filename_parts[1]);
result.to_log_index = parse<uint64_t>(filename_parts[2]);
result.extension = path.extension();
result.path = path_str;
result.path = path.generic_string();
return result;
}
@ -276,6 +275,7 @@ Changelog::Changelog(
Poco::Logger * log_,
bool compress_logs_)
: changelogs_dir(changelogs_dir_)
, changelogs_detached_dir(changelogs_dir / "detached")
, rotate_interval(rotate_interval_)
, force_sync(force_sync_)
, log(log_)
@ -288,12 +288,15 @@ Changelog::Changelog(
for (const auto & p : fs::directory_iterator(changelogs_dir))
{
if (p == changelogs_detached_dir)
continue;
auto file_description = getChangelogFileDescription(p.path());
existing_changelogs[file_description.from_log_index] = file_description;
}
if (existing_changelogs.empty())
LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", changelogs_dir);
LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", changelogs_dir.generic_string());
clean_log_thread = ThreadFromGlobalPool([this] { cleanLogThread(); });
}
@ -328,7 +331,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin
/// entries from leader.
if (changelog_description.from_log_index > last_commited_log_index && (changelog_description.from_log_index - last_commited_log_index) > 1)
{
LOG_ERROR(log, "Some records was lost, last committed log index {}, smallest available log index on disk {}. Hopefully will receive missing records from leader.", last_commited_log_index, changelog_description.from_log_index);
LOG_ERROR(log, "Some records were lost, last committed log index {}, smallest available log index on disk {}. Hopefully will receive missing records from leader.", last_commited_log_index, changelog_description.from_log_index);
/// Nothing to do with our more fresh log, leader will overwrite them, so remove everything and just start from last_commited_index
removeAllLogs();
min_log_id = last_commited_log_index;
@ -342,6 +345,12 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin
LOG_WARNING(log, "Don't have required amount of reserved log records. Need to read from {}, smallest available log index on disk {}.", start_to_read_from, changelog_description.from_log_index);
}
}
else if ((changelog_description.from_log_index - last_log_read_result->last_read_index) > 1)
{
LOG_ERROR(log, "Some records were lost, last found log index {}, while the next log index on disk is {}. Hopefully will receive missing records from leader.", last_log_read_result->last_read_index, changelog_description.from_log_index);
removeAllLogsAfter(last_log_read_result->log_start_index);
break;
}
ChangelogReader reader(changelog_description.path);
last_log_read_result = reader.readChangelog(logs, start_to_read_from, log);
@ -431,6 +440,44 @@ void Changelog::initWriter(const ChangelogFileDescription & description)
current_writer = std::make_unique<ChangelogWriter>(description.path, WriteMode::Append, description.from_log_index);
}
namespace
{
std::string getCurrentTimestampFolder()
{
const auto timestamp = LocalDateTime{std::time(nullptr)};
return fmt::format(
"{:02}{:02}{:02}T{:02}{:02}{:02}",
timestamp.year(),
timestamp.month(),
timestamp.day(),
timestamp.hour(),
timestamp.minute(),
timestamp.second());
}
}
void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end)
{
const auto timestamp_folder = changelogs_detached_dir / getCurrentTimestampFolder();
for (auto itr = begin; itr != end;)
{
if (!std::filesystem::exists(timestamp_folder))
{
LOG_WARNING(log, "Moving broken logs to {}", timestamp_folder.generic_string());
std::filesystem::create_directories(timestamp_folder);
}
LOG_WARNING(log, "Removing changelog {}", itr->second.path);
const std::filesystem::path path = itr->second.path;
const auto new_path = timestamp_folder / path.filename();
std::filesystem::rename(path, new_path);
itr = existing_changelogs.erase(itr);
}
}
void Changelog::removeAllLogsAfter(uint64_t remove_after_log_start_index)
{
auto start_to_remove_from_itr = existing_changelogs.upper_bound(remove_after_log_start_index);
@ -440,12 +487,8 @@ void Changelog::removeAllLogsAfter(uint64_t remove_after_log_start_index)
size_t start_to_remove_from_log_id = start_to_remove_from_itr->first;
/// All subsequent logs shouldn't exist. But they may exist if we crashed after writeAt started. Remove them.
for (auto itr = start_to_remove_from_itr; itr != existing_changelogs.end();)
{
LOG_WARNING(log, "Removing changelog {}, because it's goes after broken changelog entry", itr->second.path);
std::filesystem::remove(itr->second.path);
itr = existing_changelogs.erase(itr);
}
LOG_WARNING(log, "Removing changelogs that go after broken changelog entry");
removeExistingLogs(start_to_remove_from_itr, existing_changelogs.end());
std::erase_if(logs, [start_to_remove_from_log_id] (const auto & item) { return item.first >= start_to_remove_from_log_id; });
}
@ -453,12 +496,7 @@ void Changelog::removeAllLogsAfter(uint64_t remove_after_log_start_index)
void Changelog::removeAllLogs()
{
LOG_WARNING(log, "Removing all changelogs");
for (auto itr = existing_changelogs.begin(); itr != existing_changelogs.end();)
{
LOG_WARNING(log, "Removing changelog {}, because it's goes after broken changelog entry", itr->second.path);
std::filesystem::remove(itr->second.path);
itr = existing_changelogs.erase(itr);
}
removeExistingLogs(existing_changelogs.begin(), existing_changelogs.end());
logs.clear();
}

View File

@ -138,6 +138,13 @@ private:
/// Starts new file [new_start_log_index, new_start_log_index + rotate_interval]
void rotate(uint64_t new_start_log_index);
/// Currently existing changelogs
std::map<uint64_t, ChangelogFileDescription> existing_changelogs;
using ChangelogIter = decltype(existing_changelogs)::iterator;
void removeExistingLogs(ChangelogIter begin, ChangelogIter end);
static void removeLog(const std::filesystem::path & path, const std::filesystem::path & detached_folder);
/// Remove all changelogs from disk with start_index bigger than start_to_remove_from_id
void removeAllLogsAfter(uint64_t remove_after_log_start_index);
/// Remove all logs from disk
@ -148,14 +155,13 @@ private:
/// Clean useless log files in a background thread
void cleanLogThread();
const std::string changelogs_dir;
const std::filesystem::path changelogs_dir;
const std::filesystem::path changelogs_detached_dir;
const uint64_t rotate_interval;
const bool force_sync;
Poco::Logger * log;
bool compress_logs;
/// Currently existing changelogs
std::map<uint64_t, ChangelogFileDescription> existing_changelogs;
/// Current writer for changelog file
std::unique_ptr<ChangelogWriter> current_writer;

View File

@ -153,7 +153,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
/// Better to sort before serialization, otherwise snapshots can be different on different replicas
std::vector<std::pair<int64_t, Coordination::ACLs>> sorted_acl_map(snapshot.acl_map.begin(), snapshot.acl_map.end());
std::sort(sorted_acl_map.begin(), sorted_acl_map.end());
::sort(sorted_acl_map.begin(), sorted_acl_map.end());
/// Serialize ACLs map
writeBinary(sorted_acl_map.size(), out);
for (const auto & [acl_id, acls] : sorted_acl_map)
@ -195,7 +195,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
/// Session must be saved in a sorted order,
/// otherwise snapshots will be different
std::vector<std::pair<int64_t, int64_t>> sorted_session_and_timeout(snapshot.session_and_timeout.begin(), snapshot.session_and_timeout.end());
std::sort(sorted_session_and_timeout.begin(), sorted_session_and_timeout.end());
::sort(sorted_session_and_timeout.begin(), sorted_session_and_timeout.end());
/// Serialize sessions
size_t size = sorted_session_and_timeout.size();

View File

@ -698,13 +698,32 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead)
EXPECT_TRUE(fs::exists("./logs/changelog_36_40.bin" + params.extension));
}
namespace
{
void assertBrokenLogRemoved(const fs::path & log_folder, const fs::path & filename)
{
EXPECT_FALSE(fs::exists(log_folder / filename));
// broken logs are sent to the detached/{timestamp} folder
// we don't know timestamp so we iterate all of them
for (const auto & dir_entry : fs::recursive_directory_iterator(log_folder / "detached"))
{
if (dir_entry.path().filename() == filename)
return;
}
FAIL() << "Broken log " << filename << " was not moved to the detached folder";
}
}
TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
{
auto params = GetParam();
ChangelogDirTest test("./logs");
static const fs::path log_folder{"./logs"};
DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression);
auto params = GetParam();
ChangelogDirTest test(log_folder);
DB::KeeperLogStore changelog(log_folder, 5, true, params.enable_compression);
changelog.init(1, 0);
for (size_t i = 0; i < 35; ++i)
@ -736,10 +755,10 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension));
assertBrokenLogRemoved(log_folder, "changelog_16_20.bin" + params.extension);
assertBrokenLogRemoved(log_folder, "changelog_21_25.bin" + params.extension);
assertBrokenLogRemoved(log_folder, "changelog_26_30.bin" + params.extension);
assertBrokenLogRemoved(log_folder, "changelog_31_35.bin" + params.extension);
auto entry = getLogEntry("h", 7777);
changelog_reader.append(entry);
@ -751,10 +770,10 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension));
assertBrokenLogRemoved(log_folder, "changelog_16_20.bin" + params.extension);
assertBrokenLogRemoved(log_folder, "changelog_21_25.bin" + params.extension);
assertBrokenLogRemoved(log_folder, "changelog_26_30.bin" + params.extension);
assertBrokenLogRemoved(log_folder, "changelog_31_35.bin" + params.extension);
DB::KeeperLogStore changelog_reader2("./logs", 5, true, params.enable_compression);
changelog_reader2.init(1, 0);
@ -788,14 +807,13 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2)
EXPECT_EQ(changelog_reader.size(), 0);
EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_21_40.bin" + params.extension));
assertBrokenLogRemoved("./logs", "changelog_21_40.bin" + params.extension);
auto entry = getLogEntry("hello_world", 7777);
changelog_reader.append(entry);
changelog_reader.end_of_append_batch(0, 0);
EXPECT_EQ(changelog_reader.size(), 1);
EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777);
DB::KeeperLogStore changelog_reader2("./logs", 1, true, params.enable_compression);
changelog_reader2.init(1, 0);
EXPECT_EQ(changelog_reader2.size(), 1);
@ -825,10 +843,40 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles)
DB::KeeperLogStore changelog_reader("./logs", 20, true, params.enable_compression);
/// It should print error message, but still able to start
changelog_reader.init(5, 0);
EXPECT_FALSE(fs::exists("./logs/changelog_1_20.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_21_40.bin" + params.extension));
assertBrokenLogRemoved("./logs", "changelog_21_40.bin" + params.extension);
}
TEST_P(CoordinationTest, ChangelogTestLostFiles2)
{
auto params = GetParam();
ChangelogDirTest test("./logs");
DB::KeeperLogStore changelog("./logs", 10, true, params.enable_compression);
changelog.init(1, 0);
for (size_t i = 0; i < 35; ++i)
{
auto entry = getLogEntry(std::to_string(i) + "_hello_world", (i + 44) * 10);
changelog.append(entry);
}
changelog.end_of_append_batch(0, 0);
EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_20.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_21_30.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_31_40.bin" + params.extension));
// we have a gap in our logs, we need to remove all the logs after the gap
fs::remove("./logs/changelog_21_30.bin" + params.extension);
DB::KeeperLogStore changelog_reader("./logs", 10, true, params.enable_compression);
/// It should print error message, but still able to start
changelog_reader.init(5, 0);
EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_20.bin" + params.extension));
assertBrokenLogRemoved("./logs", "changelog_31_40.bin" + params.extension);
}
struct IntNode
{
int value;

View File

@ -693,6 +693,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \
M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \
M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \
M(Bool, input_format_avro_null_as_default, false, "For Avro/AvroConfluent format: insert default in case of null and non Nullable column", 0) \
M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \
\
M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \

View File

@ -1,3 +1,4 @@
#include <cstddef>
#include <Columns/IColumn.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnSparse.h>
@ -162,6 +163,12 @@ void IDataType::insertDefaultInto(IColumn & column) const
column.insertDefault();
}
void IDataType::insertManyDefaultsInto(IColumn & column, size_t n) const
{
for (size_t i = 0; i < n; ++i)
insertDefaultInto(column);
}
void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const
{
/// replace only if not null

View File

@ -159,6 +159,8 @@ public:
*/
virtual void insertDefaultInto(IColumn & column) const;
void insertManyDefaultsInto(IColumn & column, size_t n) const;
/// Checks that two instances belong to the same type
virtual bool equals(const IDataType & rhs) const = 0;

View File

@ -455,7 +455,7 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node * node)
}
/// Sort to always create the same type for the same set of subcolumns.
std::sort(tuple_elements.begin(), tuple_elements.end(),
::sort(tuple_elements.begin(), tuple_elements.end(),
[](const auto & lhs, const auto & rhs) { return std::get<0>(lhs) < std::get<0>(rhs); });
auto tuple_names = extractVector<0>(tuple_elements);
@ -692,7 +692,7 @@ void replaceMissedSubcolumnsByConstants(
res.emplace_back(full_name, types[i]);
}
std::sort(res.begin(), res.end());
::sort(res.begin(), res.end());
return res;
};

View File

@ -176,6 +176,11 @@ Poco::Timestamp DiskDecorator::getLastModified(const String & path) const
return delegate->getLastModified(path);
}
time_t DiskDecorator::getLastChanged(const String & path) const
{
return delegate->getLastChanged(path);
}
void DiskDecorator::setReadOnly(const String & path)
{
delegate->setReadOnly(path);

View File

@ -56,6 +56,7 @@ public:
void removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override;
void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override;
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
time_t getLastChanged(const String & path) const override;
Poco::Timestamp getLastModified(const String & path) const override;
void setReadOnly(const String & path) override;
void createHardLink(const String & src_path, const String & dst_path) override;

View File

@ -198,6 +198,12 @@ public:
return delegate->getLastModified(wrapped_path);
}
time_t getLastChanged(const String & path) const override
{
auto wrapped_path = wrappedPath(path);
return delegate->getLastChanged(wrapped_path);
}
void setReadOnly(const String & path) override
{
auto wrapped_path = wrappedPath(path);

View File

@ -404,6 +404,11 @@ Poco::Timestamp DiskLocal::getLastModified(const String & path) const
return FS::getModificationTimestamp(fs::path(disk_path) / path);
}
time_t DiskLocal::getLastChanged(const String & path) const
{
return FS::getChangeTime(fs::path(disk_path) / path);
}
void DiskLocal::createHardLink(const String & src_path, const String & dst_path)
{
DB::createHardLink(fs::path(disk_path) / src_path, fs::path(disk_path) / dst_path);

View File

@ -93,6 +93,8 @@ public:
Poco::Timestamp getLastModified(const String & path) const override;
time_t getLastChanged(const String & path) const override;
void setReadOnly(const String & path) override;
void createHardLink(const String & src_path, const String & dst_path) override;

View File

@ -83,6 +83,8 @@ public:
Poco::Timestamp getLastModified(const String &) const override { return Poco::Timestamp(); }
time_t getLastChanged(const String &) const override { return {}; }
void setReadOnly(const String & path) override;
void createHardLink(const String & src_path, const String & dst_path) override;

View File

@ -100,6 +100,8 @@ public:
Poco::Timestamp getLastModified(const String &) const override { return Poco::Timestamp{}; }
time_t getLastChanged(const String &) const override { return {}; }
/// Write and modification part
std::unique_ptr<WriteBufferFromFileBase> writeFile(const String &, size_t, WriteMode, const WriteSettings &) override

View File

@ -261,6 +261,10 @@ public:
/// Get last modified time of file or directory at `path`.
virtual Poco::Timestamp getLastModified(const String & path) const = 0;
/// Get last changed time of file or directory at `path`.
/// Meaning is the same as stat.mt_ctime (e.g. different from getLastModified()).
virtual time_t getLastChanged(const String & path) const = 0;
/// Set file at `path` as read-only.
virtual void setReadOnly(const String & path) = 0;

View File

@ -376,6 +376,11 @@ Poco::Timestamp DiskObjectStorage::getLastModified(const String & path) const
return metadata_storage->getLastModified(path);
}
time_t DiskObjectStorage::getLastChanged(const String & path) const
{
return metadata_storage->getLastChanged(path);
}
void DiskObjectStorage::removeMetadata(const String & path, std::vector<String> & paths_to_remove)
{
LOG_TRACE(log, "Remove file by path: {}", backQuote(metadata_storage->getPath() + path));

View File

@ -130,6 +130,8 @@ public:
Poco::Timestamp getLastModified(const String & path) const override;
time_t getLastChanged(const String & path) const override;
bool isRemote() const override { return true; }
void shutdown() override;

View File

@ -72,7 +72,7 @@ void DiskObjectStorageMetadata::deserializeFromString(const std::string & data)
void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const
{
writeIntText(VERSION_RELATIVE_PATHS, buf);
writeIntText(VERSION_READ_ONLY_FLAG, buf);
writeChar('\n', buf);
writeIntText(remote_fs_objects.size(), buf);

View File

@ -101,6 +101,8 @@ public:
virtual Poco::Timestamp getLastModified(const std::string & path) const = 0;
virtual time_t getLastChanged(const std::string & path) const = 0;
virtual std::vector<std::string> listDirectory(const std::string & path) const = 0;
virtual DirectoryIteratorPtr iterateDirectory(const std::string & path) const = 0;

View File

@ -471,6 +471,11 @@ Poco::Timestamp MetadataStorageFromDisk::getLastModified(const std::string & pat
return disk->getLastModified(path);
}
time_t MetadataStorageFromDisk::getLastChanged(const std::string & path) const
{
return disk->getLastChanged(path);
}
uint64_t MetadataStorageFromDisk::getFileSize(const String & path) const
{
auto metadata = readMetadata(path);

View File

@ -59,6 +59,8 @@ public:
Poco::Timestamp getLastModified(const std::string & path) const override;
time_t getLastChanged(const std::string & path) const override;
std::vector<std::string> listDirectory(const std::string & path) const override;
DirectoryIteratorPtr iterateDirectory(const std::string & path) const override;

View File

@ -56,6 +56,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.avro.schema_registry_url = settings.format_avro_schema_registry_url.toString();
format_settings.avro.string_column_pattern = settings.output_format_avro_string_column_pattern.toString();
format_settings.avro.output_rows_in_file = settings.output_format_avro_rows_in_file;
format_settings.avro.null_as_default = settings.input_format_avro_null_as_default;
format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes;
format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes;
format_settings.csv.crlf_end_of_line = settings.output_format_csv_crlf_end_of_line;

View File

@ -92,6 +92,7 @@ struct FormatSettings
bool allow_missing_fields = false;
String string_column_pattern;
UInt64 output_rows_in_file = 1;
bool null_as_default = false;
} avro;
String bool_true_representation = "true";

View File

@ -13,6 +13,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionUnaryArithmetic.h>
#include <Common/FieldVisitors.h>
#include <algorithm>
@ -50,17 +51,15 @@ MutableColumnPtr buildColumnFromTernaryData(const UInt8Container & ternary_data,
const size_t rows_count = ternary_data.size();
auto new_column = ColumnUInt8::create(rows_count);
std::transform(
ternary_data.cbegin(), ternary_data.cend(), new_column->getData().begin(),
[](const auto x) { return x == Ternary::True; });
for (size_t i = 0; i < rows_count; ++i)
new_column->getData()[i] = (ternary_data[i] == Ternary::True);
if (!make_nullable)
return new_column;
auto null_column = ColumnUInt8::create(rows_count);
std::transform(
ternary_data.cbegin(), ternary_data.cend(), null_column->getData().begin(),
[](const auto x) { return x == Ternary::Null; });
for (size_t i = 0; i < rows_count; ++i)
null_column->getData()[i] = (ternary_data[i] == Ternary::Null);
return ColumnNullable::create(std::move(new_column), std::move(null_column));
}
@ -68,13 +67,14 @@ MutableColumnPtr buildColumnFromTernaryData(const UInt8Container & ternary_data,
template <typename T>
bool tryConvertColumnToBool(const IColumn * column, UInt8Container & res)
{
const auto col = checkAndGetColumn<ColumnVector<T>>(column);
if (!col)
const auto column_typed = checkAndGetColumn<ColumnVector<T>>(column);
if (!column_typed)
return false;
std::transform(
col->getData().cbegin(), col->getData().cend(), res.begin(),
[](const auto x) { return !!x; });
auto & data = column_typed->getData();
size_t data_size = data.size();
for (size_t i = 0; i < data_size; ++i)
res[i] = static_cast<bool>(data[i]);
return true;
}
@ -99,7 +99,7 @@ bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res, Func && func)
{
bool has_res = false;
for (int i = static_cast<int>(in.size()) - 1; i >= 0; --i)
for (Int64 i = static_cast<Int64>(in.size()) - 1; i >= 0; --i)
{
UInt8 x;
@ -458,7 +458,9 @@ ColumnPtr basicExecuteImpl(ColumnRawPtrs arguments, size_t input_rows_count)
for (const IColumn * column : arguments)
{
if (const auto * uint8_column = checkAndGetColumn<ColumnUInt8>(column))
{
uint8_args.push_back(uint8_column);
}
else
{
auto converted_column = ColumnUInt8::create(input_rows_count);
@ -596,14 +598,14 @@ ColumnPtr FunctionAnyArityLogical<Impl, Name>::executeShortCircuit(ColumnsWithTy
if (nulls)
applyTernaryLogic<Name>(mask, *nulls);
MutableColumnPtr res = ColumnUInt8::create();
typeid_cast<ColumnUInt8 *>(res.get())->getData() = std::move(mask);
auto res = ColumnUInt8::create();
res->getData() = std::move(mask);
if (!nulls)
return res;
MutableColumnPtr bytemap = ColumnUInt8::create();
typeid_cast<ColumnUInt8 *>(bytemap.get())->getData() = std::move(*nulls);
auto bytemap = ColumnUInt8::create();
bytemap->getData() = std::move(*nulls);
return ColumnNullable::create(std::move(res), std::move(bytemap));
}
@ -692,29 +694,14 @@ ColumnPtr FunctionAnyArityLogical<Impl, Name>::getConstantResultForNonConstArgum
return result_column;
}
template <typename A, typename Op>
struct UnaryOperationImpl
{
using ResultType = typename Op::ResultType;
using ArrayA = typename ColumnVector<A>::Container;
using ArrayC = typename ColumnVector<ResultType>::Container;
static void NO_INLINE vector(const ArrayA & a, ArrayC & c)
{
std::transform(
a.cbegin(), a.cend(), c.begin(),
[](const auto x) { return Op::apply(x); });
}
};
template <template <typename> class Impl, typename Name>
DataTypePtr FunctionUnaryLogical<Impl, Name>::getReturnTypeImpl(const DataTypes & arguments) const
{
if (!isNativeNumber(arguments[0]))
throw Exception("Illegal type ("
+ arguments[0]->getName()
+ ") of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type ({}) of argument of function {}",
arguments[0]->getName(),
getName());
return isBool(arguments[0]) ? DataTypeFactory::instance().get("Bool") : std::make_shared<DataTypeUInt8>();
}
@ -724,10 +711,9 @@ ColumnPtr functionUnaryExecuteType(const ColumnsWithTypeAndName & arguments)
{
if (auto col = checkAndGetColumn<ColumnVector<T>>(arguments[0].column.get()))
{
auto col_res = ColumnUInt8::create();
auto col_res = ColumnUInt8::create(col->getData().size());
auto & vec_res = col_res->getData();
typename ColumnUInt8::Container & vec_res = col_res->getData();
vec_res.resize(col->getData().size());
UnaryOperationImpl<T, Impl<T>>::vector(col->getData(), vec_res);
return col_res;
@ -750,9 +736,10 @@ ColumnPtr FunctionUnaryLogical<Impl, Name>::executeImpl(const ColumnsWithTypeAnd
|| (res = functionUnaryExecuteType<Impl, Int64>(arguments))
|| (res = functionUnaryExecuteType<Impl, Float32>(arguments))
|| (res = functionUnaryExecuteType<Impl, Float64>(arguments))))
throw Exception("Illegal column " + arguments[0].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of argument of function {}",
arguments[0].column->getName(),
getName());
return res;
}

View File

@ -43,6 +43,7 @@
#include <Disks/DiskRestartProxy.h>
#include <Storages/StorageDistributed.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/Freeze.h>
#include <Storages/StorageFactory.h>
#include <Parsers/ASTSystemQuery.h>
#include <Parsers/ASTDropQuery.h>
@ -235,6 +236,8 @@ BlockIO InterpreterSystemQuery::execute()
}
BlockIO result;
volume_ptr = {};
if (!query.storage_policy.empty() && !query.volume.empty())
volume_ptr = getContext()->getStoragePolicy(query.storage_policy)->getVolumeByName(query.volume);
@ -493,11 +496,18 @@ BlockIO InterpreterSystemQuery::execute()
getContext()->checkAccess(AccessType::SYSTEM_THREAD_FUZZER);
ThreadFuzzer::start();
break;
case Type::UNFREEZE:
{
getContext()->checkAccess(AccessType::SYSTEM_UNFREEZE);
/// The result contains information about deleted parts as a table. It is for compatibility with ALTER TABLE UNFREEZE query.
result = Unfreezer().unfreeze(query.backup_name, getContext());
break;
}
default:
throw Exception("Unknown type of SYSTEM query", ErrorCodes::BAD_ARGUMENTS);
}
return BlockIO();
return result;
}
void InterpreterSystemQuery::restoreReplica()
@ -968,6 +978,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
required_access.emplace_back(AccessType::SYSTEM_RESTART_DISK);
break;
}
case Type::UNFREEZE:
{
required_access.emplace_back(AccessType::SYSTEM_UNFREEZE);
break;
}
case Type::STOP_LISTEN_QUERIES:
case Type::START_LISTEN_QUERIES:
case Type::STOP_THREAD_FUZZER:

View File

@ -201,7 +201,7 @@ void TransactionLog::loadLogFromZooKeeper()
/// 3. support 64-bit CSNs on top of Apache ZooKeeper (it uses Int32 for sequential numbers)
Strings entries_list = zookeeper->getChildren(zookeeper_path_log, nullptr, log_updated_event);
chassert(!entries_list.empty());
std::sort(entries_list.begin(), entries_list.end());
::sort(entries_list.begin(), entries_list.end());
loadEntries(entries_list.begin(), entries_list.end());
chassert(!last_loaded_entry.empty());
chassert(latest_snapshot == deserializeCSN(last_loaded_entry));
@ -262,7 +262,7 @@ void TransactionLog::loadNewEntries()
{
Strings entries_list = zookeeper->getChildren(zookeeper_path_log, nullptr, log_updated_event);
chassert(!entries_list.empty());
std::sort(entries_list.begin(), entries_list.end());
::sort(entries_list.begin(), entries_list.end());
auto it = std::upper_bound(entries_list.begin(), entries_list.end(), last_loaded_entry);
loadEntries(it, entries_list.end());
chassert(last_loaded_entry == entries_list.back());
@ -602,7 +602,7 @@ void TransactionLog::sync() const
{
Strings entries_list = zookeeper->getChildren(zookeeper_path_log);
chassert(!entries_list.empty());
std::sort(entries_list.begin(), entries_list.end());
::sort(entries_list.begin(), entries_list.end());
CSN newest_csn = deserializeCSN(entries_list.back());
waitForCSNLoaded(newest_csn);
}

View File

@ -469,7 +469,7 @@ void removeUnneededColumnsFromSelectClause(ASTSelectQuery * select_query, const
for (const auto & name : required_result_columns)
name_pos[name] = pos++;
}
std::sort(elements.begin(), elements.end(), [&](const auto & lhs, const auto & rhs)
::sort(elements.begin(), elements.end(), [&](const auto & lhs, const auto & rhs)
{
String lhs_name = lhs->getAliasOrColumnName();
String rhs_name = rhs->getAliasOrColumnName();

View File

@ -66,6 +66,7 @@ public:
START_DISTRIBUTED_SENDS,
START_THREAD_FUZZER,
STOP_THREAD_FUZZER,
UNFREEZE,
END
};
@ -93,6 +94,7 @@ public:
UInt64 seconds{};
String filesystem_cache_path;
String backup_name;
String getID(char) const override { return "SYSTEM query"; }

View File

@ -363,6 +363,20 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
break;
}
case Type::UNFREEZE:
{
ASTPtr ast;
if (ParserKeyword{"WITH NAME"}.ignore(pos, expected) && ParserStringLiteral{}.parse(pos, ast, expected))
{
res->backup_name = ast->as<ASTLiteral &>().value.get<const String &>();
}
else
{
return false;
}
break;
}
default:
{
parseQueryWithOnCluster(res, pos, expected);

View File

@ -4,7 +4,6 @@
#include <numeric>
#include <Core/Defines.h>
#include <Core/Field.h>
#include <Common/LRUCache.h>
@ -13,12 +12,9 @@
#include <IO/ReadHelpers.h>
#include <IO/HTTPCommon.h>
#include <Formats/verbosePrintString.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeFixedString.h>
@ -28,7 +24,7 @@
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/getLeastSupertype.h>
#include <DataTypes/DataTypeMap.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnFixedString.h>
@ -36,33 +32,24 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnMap.h>
#include <avro/Compiler.hh>
#include <avro/DataFile.hh>
#include <avro/Decoder.hh>
#include <avro/Encoder.hh>
#include <avro/Generic.hh>
#include <avro/GenericDatum.hh>
#include <avro/Node.hh>
#include <avro/NodeConcepts.hh>
#include <avro/NodeImpl.hh>
#include <avro/Reader.hh>
#include <avro/Schema.hh>
#include <avro/Specific.hh>
#include <avro/Types.hh>
#include <avro/ValidSchema.hh>
#include <avro/Writer.hh>
#include <Compiler.hh>
#include <DataFile.hh>
#include <Decoder.hh>
#include <Node.hh>
#include <NodeConcepts.hh>
#include <NodeImpl.hh>
#include <Types.hh>
#include <ValidSchema.hh>
#include <Poco/BinaryReader.h>
#include <Poco/Buffer.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <Poco/JSON/Parser.h>
#include <Poco/MemoryStream.h>
#include <Poco/Net/HTTPClientSession.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Net/HTTPResponse.h>
#include <Poco/Poco.h>
#include <Poco/URI.h>
@ -292,30 +279,42 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node
break;
case avro::AVRO_UNION:
{
auto nullable_deserializer = [root_node, target_type](size_t non_null_union_index)
if (root_node->leaves() == 2
&& (root_node->leafAt(0)->type() == avro::AVRO_NULL || root_node->leafAt(1)->type() == avro::AVRO_NULL))
{
auto nested_deserialize = createDeserializeFn(root_node->leafAt(non_null_union_index), removeNullable(target_type));
return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder)
size_t non_null_union_index = root_node->leafAt(0)->type() == avro::AVRO_NULL ? 1 : 0;
if (target.isNullable())
{
ColumnNullable & col = assert_cast<ColumnNullable &>(column);
size_t union_index = decoder.decodeUnionIndex();
if (union_index == non_null_union_index)
auto nested_deserialize = this->createDeserializeFn(root_node->leafAt(non_null_union_index), removeNullable(target_type));
return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder)
{
nested_deserialize(col.getNestedColumn(), decoder);
col.getNullMapData().push_back(0);
}
else
ColumnNullable & col = assert_cast<ColumnNullable &>(column);
size_t union_index = decoder.decodeUnionIndex();
if (union_index == non_null_union_index)
{
nested_deserialize(col.getNestedColumn(), decoder);
col.getNullMapData().push_back(0);
}
else
{
col.insertDefault();
}
};
}
if (null_as_default)
{
auto nested_deserialize = this->createDeserializeFn(root_node->leafAt(non_null_union_index), target_type);
return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder)
{
col.insertDefault();
}
};
};
if (root_node->leaves() == 2 && target.isNullable())
{
if (root_node->leafAt(0)->type() == avro::AVRO_NULL)
return nullable_deserializer(1);
if (root_node->leafAt(1)->type() == avro::AVRO_NULL)
return nullable_deserializer(0);
size_t union_index = decoder.decodeUnionIndex();
if (union_index == non_null_union_index)
nested_deserialize(column, decoder);
else
column.insertDefault();
};
}
}
break;
}
@ -386,8 +385,69 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node
}
case avro::AVRO_SYMBOLIC:
return createDeserializeFn(avro::resolveSymbol(root_node), target_type);
case avro::AVRO_MAP:
case avro::AVRO_RECORD:
{
if (target.isTuple())
{
const DataTypeTuple & tuple_type = assert_cast<const DataTypeTuple &>(*target_type);
const auto & nested_types = tuple_type.getElements();
std::vector<std::pair<DeserializeFn, size_t>> nested_deserializers;
nested_deserializers.reserve(root_node->leaves());
if (root_node->leaves() != nested_types.size())
throw Exception(ErrorCodes::INCORRECT_DATA, "The number of leaves in record doesn't match the number of elements in tuple");
for (size_t i = 0; i != root_node->leaves(); ++i)
{
const auto & name = root_node->nameAt(i);
size_t pos = tuple_type.getPositionByName(name);
auto nested_deserializer = createDeserializeFn(root_node->leafAt(i), nested_types[pos]);
nested_deserializers.emplace_back(nested_deserializer, pos);
}
return [nested_deserializers](IColumn & column, avro::Decoder & decoder)
{
ColumnTuple & column_tuple = assert_cast<ColumnTuple &>(column);
auto nested_columns = column_tuple.getColumns();
for (const auto & [nested_deserializer, pos] : nested_deserializers)
nested_deserializer(*nested_columns[pos], decoder);
};
}
break;
}
case avro::AVRO_MAP:
{
if (target.isMap())
{
const auto & map_type = assert_cast<const DataTypeMap &>(*target_type);
const auto & keys_type = map_type.getKeyType();
const auto & values_type = map_type.getValueType();
auto keys_source_type = root_node->leafAt(0);
auto values_source_type = root_node->leafAt(1);
auto keys_deserializer = createDeserializeFn(keys_source_type, keys_type);
auto values_deserializer = createDeserializeFn(values_source_type, values_type);
return [keys_deserializer, values_deserializer](IColumn & column, avro::Decoder & decoder)
{
ColumnMap & column_map = assert_cast<ColumnMap &>(column);
ColumnArray & column_array = column_map.getNestedColumn();
ColumnArray::Offsets & offsets = column_array.getOffsets();
ColumnTuple & nested_columns = column_map.getNestedData();
IColumn & keys_column = nested_columns.getColumn(0);
IColumn & values_column = nested_columns.getColumn(1);
size_t total = 0;
for (size_t n = decoder.mapStart(); n != 0; n = decoder.mapNext())
{
total += n;
for (size_t i = 0; i < n; ++i)
{
keys_deserializer(keys_column, decoder);
values_deserializer(values_column, decoder);
}
}
offsets.push_back(offsets.back() + total);
};
}
break;
}
default:
break;
}
@ -577,7 +637,8 @@ AvroDeserializer::Action AvroDeserializer::createAction(const Block & header, co
}
}
AvroDeserializer::AvroDeserializer(const Block & header, avro::ValidSchema schema, bool allow_missing_fields)
AvroDeserializer::AvroDeserializer(const Block & header, avro::ValidSchema schema, bool allow_missing_fields, bool null_as_default_)
: null_as_default(null_as_default_)
{
const auto & schema_root = schema.root();
if (schema_root->type() != avro::AVRO_RECORD)
@ -615,15 +676,15 @@ void AvroDeserializer::deserializeRow(MutableColumns & columns, avro::Decoder &
AvroRowInputFormat::AvroRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_)
: IRowInputFormat(header_, in_, params_),
allow_missing_fields(format_settings_.avro.allow_missing_fields)
: IRowInputFormat(header_, in_, params_), format_settings(format_settings_)
{
}
void AvroRowInputFormat::readPrefix()
{
file_reader_ptr = std::make_unique<avro::DataFileReaderBase>(std::make_unique<InputStreamReadBufferAdapter>(*in));
deserializer_ptr = std::make_unique<AvroDeserializer>(output.getHeader(), file_reader_ptr->dataSchema(), allow_missing_fields);
deserializer_ptr = std::make_unique<AvroDeserializer>(
output.getHeader(), file_reader_ptr->dataSchema(), format_settings.avro.allow_missing_fields, format_settings.avro.null_as_default);
file_reader_ptr->init();
}
@ -809,7 +870,8 @@ const AvroDeserializer & AvroConfluentRowInputFormat::getOrCreateDeserializer(Sc
if (it == deserializer_cache.end())
{
auto schema = schema_registry->getSchema(schema_id);
AvroDeserializer deserializer(output.getHeader(), schema, format_settings.avro.allow_missing_fields);
AvroDeserializer deserializer(
output.getHeader(), schema, format_settings.avro.allow_missing_fields, format_settings.avro.null_as_default);
it = deserializer_cache.emplace(schema_id, deserializer).first;
}
return it->second;
@ -891,11 +953,27 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node)
if (node->leaves() == 2 && (node->leafAt(0)->type() == avro::Type::AVRO_NULL || node->leafAt(1)->type() == avro::Type::AVRO_NULL))
{
size_t nested_leaf_index = node->leafAt(0)->type() == avro::Type::AVRO_NULL ? 1 : 0;
return makeNullable(avroNodeToDataType(node->leafAt(nested_leaf_index)));
auto nested_type = avroNodeToDataType(node->leafAt(nested_leaf_index));
return nested_type->canBeInsideNullable() ? makeNullable(nested_type) : nested_type;
}
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro type UNION is not supported for inserting.");
case avro::Type::AVRO_SYMBOLIC:
return avroNodeToDataType(avro::resolveSymbol(node));
case avro::Type::AVRO_RECORD:
{
DataTypes nested_types;
nested_types.reserve(node->leaves());
Names nested_names;
nested_names.reserve(node->leaves());
for (size_t i = 0; i != node->leaves(); ++i)
{
nested_types.push_back(avroNodeToDataType(node->leafAt(i)));
nested_names.push_back(node->nameAt(i));
}
return std::make_shared<DataTypeTuple>(nested_types, nested_names);
}
case avro::Type::AVRO_MAP:
return std::make_shared<DataTypeMap>(avroNodeToDataType(node->leafAt(0)), avroNodeToDataType(node->leafAt(1)));
default:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro column {} is not supported for inserting.");
}

View File

@ -15,10 +15,10 @@
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <avro/DataFile.hh>
#include <avro/Decoder.hh>
#include <avro/Schema.hh>
#include <avro/ValidSchema.hh>
#include <DataFile.hh>
#include <Decoder.hh>
#include <Schema.hh>
#include <ValidSchema.hh>
namespace DB
@ -32,13 +32,13 @@ namespace ErrorCodes
class AvroDeserializer
{
public:
AvroDeserializer(const Block & header, avro::ValidSchema schema, bool allow_missing_fields);
AvroDeserializer(const Block & header, avro::ValidSchema schema, bool allow_missing_fields, bool null_as_default_);
void deserializeRow(MutableColumns & columns, avro::Decoder & decoder, RowReadExtension & ext) const;
private:
using DeserializeFn = std::function<void(IColumn & column, avro::Decoder & decoder)>;
using SkipFn = std::function<void(avro::Decoder & decoder)>;
static DeserializeFn createDeserializeFn(avro::NodePtr root_node, DataTypePtr target_type);
DeserializeFn createDeserializeFn(avro::NodePtr root_node, DataTypePtr target_type);
SkipFn createSkipFn(avro::NodePtr root_node);
struct Action
@ -113,6 +113,8 @@ private:
/// Map from name of named Avro type (record, enum, fixed) to SkipFn.
/// This is to avoid infinite recursion when Avro schema contains self-references. e.g. LinkedList
std::map<avro::Name, SkipFn> symbolic_skip_fn_map;
bool null_as_default = false;
};
class AvroRowInputFormat final : public IRowInputFormat
@ -128,7 +130,7 @@ private:
std::unique_ptr<avro::DataFileReaderBase> file_reader_ptr;
std::unique_ptr<AvroDeserializer> deserializer_ptr;
bool allow_missing_fields;
FormatSettings format_settings;
};
/// Confluent framing + Avro binary datum encoding. Mainly used for Kafka.

View File

@ -1,23 +1,21 @@
#include "AvroRowOutputFormat.h"
#if USE_AVRO
#include <Core/Defines.h>
#include <Core/Field.h>
#include <IO/Operators.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Formats/verbosePrintString.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeMap.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnFixedString.h>
@ -25,21 +23,13 @@
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnMap.h>
#include <avro/Compiler.hh>
#include <avro/DataFile.hh>
#include <avro/Decoder.hh>
#include <avro/Encoder.hh>
#include <avro/Generic.hh>
#include <avro/GenericDatum.hh>
#include <avro/Node.hh>
#include <avro/NodeConcepts.hh>
#include <avro/NodeImpl.hh>
#include <avro/Reader.hh>
#include <avro/Schema.hh>
#include <avro/Specific.hh>
#include <avro/ValidSchema.hh>
#include <avro/Writer.hh>
#include <DataFile.hh>
#include <Encoder.hh>
#include <Node.hh>
#include <Schema.hh>
#include <re2/re2.h>
@ -321,6 +311,70 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF
}
case TypeIndex::Nothing:
return {avro::NullSchema(), [](const IColumn &, size_t, avro::Encoder & encoder) { encoder.encodeNull(); }};
case TypeIndex::Tuple:
{
const auto & tuple_type = assert_cast<const DataTypeTuple &>(*data_type);
const auto & nested_types = tuple_type.getElements();
const auto & nested_names = tuple_type.getElementNames();
std::vector<SerializeFn> nested_serializers;
nested_serializers.reserve(nested_types.size());
auto schema = avro::RecordSchema(column_name);
for (size_t i = 0; i != nested_types.size(); ++i)
{
auto nested_mapping = createSchemaWithSerializeFn(nested_types[i], type_name_increment, nested_names[i]);
schema.addField(nested_names[i], nested_mapping.schema);
nested_serializers.push_back(nested_mapping.serialize);
}
return {schema, [nested_serializers](const IColumn & column, size_t row_num, avro::Encoder & encoder)
{
const ColumnTuple & column_tuple = assert_cast<const ColumnTuple &>(column);
const auto & nested_columns = column_tuple.getColumns();
for (size_t i = 0; i != nested_serializers.size(); ++i)
nested_serializers[i](*nested_columns[i], row_num, encoder);
}};
}
case TypeIndex::Map:
{
const auto & map_type = assert_cast<const DataTypeMap &>(*data_type);
const auto & keys_type = map_type.getKeyType();
if (!isStringOrFixedString(keys_type))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro Maps support only keys with type String, got {}", keys_type->getName());
auto keys_serializer = [](const IColumn & column, size_t row_num, avro::Encoder & encoder)
{
const StringRef & s = column.getDataAt(row_num);
encoder.encodeString(s.toString());
};
const auto & values_type = map_type.getValueType();
auto values_mapping = createSchemaWithSerializeFn(values_type, type_name_increment, column_name + ".value");
auto schema = avro::MapSchema(values_mapping.schema);
return {schema, [keys_serializer, values_mapping](const IColumn & column, size_t row_num, avro::Encoder & encoder)
{
const ColumnMap & column_map = assert_cast<const ColumnMap &>(column);
const ColumnArray & column_array = column_map.getNestedColumn();
const ColumnArray::Offsets & offsets = column_array.getOffsets();
size_t offset = offsets[row_num - 1];
size_t next_offset = offsets[row_num];
size_t row_count = next_offset - offset;
const ColumnTuple & nested_columns = column_map.getNestedData();
const IColumn & keys_column = nested_columns.getColumn(0);
const IColumn & values_column = nested_columns.getColumn(1);
encoder.mapStart();
if (row_count > 0)
encoder.setItemCount(row_count);
for (size_t i = offset; i < next_offset; ++i)
{
keys_serializer(keys_column, i, encoder);
values_mapping.serialize(values_column, i, encoder);
}
encoder.mapEnd();
}};
}
default:
break;
}

View File

@ -9,9 +9,9 @@
#include <IO/WriteBuffer.h>
#include <Processors/Formats/IRowOutputFormat.h>
#include <avro/DataFile.hh>
#include <avro/Schema.hh>
#include <avro/ValidSchema.hh>
#include <DataFile.hh>
#include <Schema.hh>
#include <ValidSchema.hh>
namespace DB

View File

@ -1,5 +1,4 @@
#include <Processors/Merges/IMergingTransform.h>
#include <Processors/Transforms/SelectorInfo.h>
namespace DB
{
@ -181,68 +180,4 @@ IProcessor::Status IMergingTransformBase::prepare()
return Status::Ready;
}
static void filterChunk(IMergingAlgorithm::Input & input, size_t selector_position)
{
if (!input.chunk.getChunkInfo())
throw Exception("IMergingTransformBase expected ChunkInfo for input chunk", ErrorCodes::LOGICAL_ERROR);
const auto * chunk_info = typeid_cast<const SelectorInfo *>(input.chunk.getChunkInfo().get());
if (!chunk_info)
throw Exception("IMergingTransformBase expected SelectorInfo for input chunk", ErrorCodes::LOGICAL_ERROR);
const auto & selector = chunk_info->selector;
IColumn::Filter filter;
filter.resize_fill(selector.size());
size_t num_rows = input.chunk.getNumRows();
auto columns = input.chunk.detachColumns();
size_t num_result_rows = 0;
for (size_t row = 0; row < num_rows; ++row)
{
if (selector[row] == selector_position)
{
++num_result_rows;
filter[row] = 1;
}
}
if (!filter.empty() && filter.back() == 0)
{
filter.back() = 1;
++num_result_rows;
input.skip_last_row = true;
}
for (auto & column : columns)
column = column->filter(filter, num_result_rows);
input.chunk.clear();
input.chunk.setColumns(std::move(columns), num_result_rows);
}
void IMergingTransformBase::filterChunks()
{
if (state.selector_position < 0)
return;
if (!state.init_chunks.empty())
{
for (size_t i = 0; i < input_states.size(); ++i)
{
auto & input = state.init_chunks[i];
if (!input.chunk)
continue;
filterChunk(input, state.selector_position);
}
}
if (state.has_input)
filterChunk(state.input_chunk, state.selector_position);
}
}

View File

@ -28,17 +28,10 @@ public:
Status prepare() override;
/// Set position which will be used in selector if input chunk has attached SelectorInfo (see SelectorInfo.h).
/// Columns will be filtered, keep only rows labeled with this position.
/// It is used in parallel final.
void setSelectorPosition(size_t position) { state.selector_position = position; }
protected:
virtual void onNewInput(); /// Is called when new input is added. Only if have_all_inputs = false.
virtual void onFinish() {} /// Is called when all data is processed.
void filterChunks(); /// Filter chunks if selector position was set. For parallel final.
/// Processor state.
struct State
{
@ -50,7 +43,6 @@ protected:
size_t next_input_to_read = 0;
IMergingAlgorithm::Inputs init_chunks;
ssize_t selector_position = -1;
};
State state;
@ -92,8 +84,6 @@ public:
void work() override
{
filterChunks();
if (!state.init_chunks.empty())
algorithm.initialize(std::move(state.init_chunks));

View File

@ -241,7 +241,9 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
if (missign_column_index < missing_columns.size() && missing_columns[missign_column_index] == i)
{
++missign_column_index;
auto column = ColumnConst::create(col.column->cloneResized(1), 0);
auto column_with_default = col.column->cloneEmpty();
col.type->insertDefaultInto(*column_with_default);
auto column = ColumnConst::create(std::move(column_with_default), 0);
const auto * node = &dag->addColumn({ColumnPtr(std::move(column)), col.type, col.name});
node = &dag->materializeNode(*node);
index.push_back(node);

View File

@ -86,6 +86,9 @@ static void doDescribeProcessor(const IProcessor & processor, size_t count, IQue
doDescribeHeader(*last_header, num_equal_headers, settings);
}
if (!processor.getDescription().empty())
settings.out << String(settings.offset, settings.indent_char) << "Description: " << processor.getDescription() << '\n';
settings.offset += settings.indent;
}

View File

@ -0,0 +1,274 @@
#include <algorithm>
#include <memory>
#include <numeric>
#include <queue>
#include <unordered_map>
#include <vector>
#include <Core/Field.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Processors/QueryPlan/PartsSplitter.h>
#include <Processors/Transforms/FilterSortedStreamByRange.h>
#include <Storages/MergeTree/RangesInDataPart.h>
using namespace DB;
namespace
{
using Value = std::vector<Field>;
std::string toString(const Value & value)
{
return fmt::format("({})", fmt::join(value, ", "));
}
/// Adaptor to access PK values from index.
class IndexAccess
{
public:
explicit IndexAccess(const RangesInDataParts & parts_) : parts(parts_) { }
Value getValue(size_t part_idx, size_t mark) const
{
const auto & index = parts[part_idx].data_part->index;
Value value(index.size());
for (size_t i = 0; i < value.size(); ++i)
index[i]->get(mark, value[i]);
return value;
}
size_t getMarkRows(size_t part_idx, size_t mark) const { return parts[part_idx].data_part->index_granularity.getMarkRows(mark); }
size_t getTotalRowCount() const
{
size_t total = 0;
for (const auto & part : parts)
total += part.getRowsCount();
return total;
}
private:
const RangesInDataParts & parts;
};
/// Splits parts into layers, each layer will contain parts subranges with PK values from its own range.
/// Will try to produce exactly max_layer layers but may return less if data is distributed in not a very parallelizable way.
std::pair<std::vector<Value>, std::vector<RangesInDataParts>> split(RangesInDataParts parts, size_t max_layers)
{
// We will advance the iterator pointing to the mark with the smallest PK value until there will be not less than rows_per_layer rows in the current layer (roughly speaking).
// Then we choose the last observed value as the new border, so the current layer will consists of granules with values greater than the previous mark and less or equal
// than the new border.
struct PartsRangesIterator
{
struct RangeInDataPart : MarkRange
{
size_t part_idx;
};
enum class EventType
{
RangeBeginning,
RangeEnding,
};
bool operator<(const PartsRangesIterator & other) const { return std::tie(value, event) > std::tie(other.value, other.event); }
Value value;
RangeInDataPart range;
EventType event;
};
const auto index_access = std::make_unique<IndexAccess>(parts);
std::priority_queue<PartsRangesIterator> parts_ranges_queue;
for (size_t part_idx = 0; part_idx < parts.size(); ++part_idx)
{
for (const auto & range : parts[part_idx].ranges)
{
parts_ranges_queue.push(
{index_access->getValue(part_idx, range.begin), {range, part_idx}, PartsRangesIterator::EventType::RangeBeginning});
const auto & index_granularity = parts[part_idx].data_part->index_granularity;
if (index_granularity.hasFinalMark() && range.end + 1 == index_granularity.getMarksCount())
parts_ranges_queue.push(
{index_access->getValue(part_idx, range.end), {range, part_idx}, PartsRangesIterator::EventType::RangeEnding});
}
}
/// The beginning of currently started (but not yet finished) range of marks of a part in the current layer.
std::unordered_map<size_t, size_t> current_part_range_begin;
/// The current ending of a range of marks of a part in the current layer.
std::unordered_map<size_t, size_t> current_part_range_end;
/// Determine borders between layers.
std::vector<Value> borders;
std::vector<RangesInDataParts> result_layers;
const size_t rows_per_layer = std::max<size_t>(index_access->getTotalRowCount() / max_layers, 1);
while (!parts_ranges_queue.empty())
{
// New layer should include last granules of still open ranges from the previous layer,
// because they may already contain values greater than the last border.
size_t rows_in_current_layer = 0;
size_t marks_in_current_layer = 0;
// Intersection between the current and next layers is just the last observed marks of each still open part range. Ratio is empirical.
auto layers_intersection_is_too_big = [&]()
{
const auto intersected_parts = current_part_range_end.size();
return marks_in_current_layer < intersected_parts * 2;
};
result_layers.emplace_back();
while (rows_in_current_layer < rows_per_layer || layers_intersection_is_too_big() || result_layers.size() == max_layers)
{
// We're advancing iterators until a new value showed up.
Value last_value;
while (!parts_ranges_queue.empty() && (last_value.empty() || last_value == parts_ranges_queue.top().value))
{
auto current = parts_ranges_queue.top();
parts_ranges_queue.pop();
const auto part_idx = current.range.part_idx;
if (current.event == PartsRangesIterator::EventType::RangeEnding)
{
result_layers.back().emplace_back(
parts[part_idx].data_part,
parts[part_idx].part_index_in_query,
MarkRanges{{current_part_range_begin[part_idx], current.range.end}});
current_part_range_begin.erase(part_idx);
current_part_range_end.erase(part_idx);
continue;
}
last_value = std::move(current.value);
rows_in_current_layer += index_access->getMarkRows(part_idx, current.range.begin);
marks_in_current_layer++;
current_part_range_begin.try_emplace(part_idx, current.range.begin);
current_part_range_end[part_idx] = current.range.begin;
if (current.range.begin + 1 < current.range.end)
{
current.range.begin++;
current.value = index_access->getValue(part_idx, current.range.begin);
parts_ranges_queue.push(std::move(current));
}
}
if (parts_ranges_queue.empty())
break;
if (rows_in_current_layer >= rows_per_layer && !layers_intersection_is_too_big() && result_layers.size() < max_layers)
borders.push_back(last_value);
}
for (const auto & [part_idx, last_mark] : current_part_range_end)
{
result_layers.back().emplace_back(
parts[part_idx].data_part,
parts[part_idx].part_index_in_query,
MarkRanges{{current_part_range_begin[part_idx], last_mark + 1}});
current_part_range_begin[part_idx] = current_part_range_end[part_idx];
}
}
for (auto & layer : result_layers)
{
std::stable_sort(
layer.begin(),
layer.end(),
[](const auto & lhs, const auto & rhs) { return lhs.part_index_in_query < rhs.part_index_in_query; });
}
return std::make_pair(std::move(borders), std::move(result_layers));
}
/// Will return borders.size()+1 filters in total, i-th filter will accept rows with PK values within the range [borders[i-1], borders[i]).
std::vector<ASTPtr> buildFilters(const KeyDescription & primary_key, const std::vector<Value> & borders)
{
auto add_and_condition = [&](ASTPtr & result, const ASTPtr & foo) { result = !result ? foo : makeASTFunction("and", result, foo); };
/// Produces ASTPtr to predicate (pk_col0, pk_col1, ... , pk_colN) > (value[0], value[1], ... , value[N])
auto lexicographically_greater = [&](const Value & value)
{
// PK may contain functions of the table columns, so we need the actual PK AST with all expressions it contains.
ASTPtr pk_columns_as_tuple = makeASTFunction("tuple", primary_key.expression_list_ast->children);
ASTPtr value_ast = std::make_shared<ASTExpressionList>();
for (size_t i = 0; i < value.size(); ++i)
{
const auto & types = primary_key.data_types;
ASTPtr component_ast = std::make_shared<ASTLiteral>(value[i]);
// Values of some types (e.g. Date, DateTime) are stored in columns as numbers and we get them as just numbers from the index.
// So we need an explicit Cast for them.
if (isColumnedAsNumber(types.at(i)->getTypeId()) && !isNumber(types.at(i)->getTypeId()))
component_ast = makeASTFunction("cast", std::move(component_ast), std::make_shared<ASTLiteral>(types.at(i)->getName()));
value_ast->children.push_back(std::move(component_ast));
}
ASTPtr value_as_tuple = makeASTFunction("tuple", value_ast->children);
return makeASTFunction("greater", pk_columns_as_tuple, value_as_tuple);
};
std::vector<ASTPtr> filters(borders.size() + 1);
for (size_t layer = 0; layer <= borders.size(); ++layer)
{
if (layer > 0)
add_and_condition(filters[layer], lexicographically_greater(borders[layer - 1]));
if (layer < borders.size())
add_and_condition(filters[layer], makeASTFunction("not", lexicographically_greater(borders[layer])));
}
return filters;
}
}
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
Pipes buildPipesForReadingByPKRanges(
const KeyDescription & primary_key,
RangesInDataParts parts,
size_t max_layers,
ContextPtr context,
ReadingInOrderStepGetter && reading_step_getter)
{
if (max_layers <= 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "max_layer should be greater than 1.");
auto && [borders, result_layers] = split(std::move(parts), max_layers);
auto filters = buildFilters(primary_key, borders);
Pipes pipes(result_layers.size());
for (size_t i = 0; i < result_layers.size(); ++i)
{
pipes[i] = reading_step_getter(std::move(result_layers[i]));
auto & filter_function = filters[i];
if (!filter_function)
continue;
auto syntax_result = TreeRewriter(context).analyze(filter_function, primary_key.expression->getRequiredColumnsWithTypes());
auto actions = ExpressionAnalyzer(filter_function, syntax_result, context).getActionsDAG(false);
ExpressionActionsPtr expression_actions = std::make_shared<ExpressionActions>(std::move(actions));
auto description = fmt::format(
"filter values in [{}, {})", i ? ::toString(borders[i - 1]) : "-inf", i < borders.size() ? ::toString(borders[i]) : "+inf");
auto pk_expression = std::make_shared<ExpressionActions>(primary_key.expression->getActionsDAG().clone());
pipes[i].addSimpleTransform([pk_expression](const Block & header)
{ return std::make_shared<ExpressionTransform>(header, pk_expression); });
pipes[i].addSimpleTransform(
[&](const Block & header)
{
auto step = std::make_shared<FilterSortedStreamByRange>(header, expression_actions, filter_function->getColumnName(), true);
step->setDescription(description);
return step;
});
}
return pipes;
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <functional>
#include <Interpreters/Context_fwd.h>
#include <QueryPipeline/Pipe.h>
#include <Storages/KeyDescription.h>
#include <Storages/MergeTree/RangesInDataPart.h>
namespace DB
{
using ReadingInOrderStepGetter = std::function<Pipe(RangesInDataParts)>;
/// Splits parts into layers, each layer will contain parts subranges with PK values from its own range.
/// A separate pipe will be constructed for each layer with a reading step (provided by the reading_step_getter) and a filter for this layer's range of PK values.
/// Will try to produce exactly max_layer pipes but may return less if data is distributed in not a very parallelizable way.
Pipes buildPipesForReadingByPKRanges(
const KeyDescription & primary_key,
RangesInDataParts parts,
size_t max_layers,
ContextPtr context,
ReadingInOrderStepGetter && reading_step_getter);
}

View File

@ -1,14 +1,16 @@
#include <algorithm>
#include <functional>
#include <memory>
#include <numeric>
#include <queue>
#include <stdexcept>
#include <IO/Operators.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Processors/ConcatProcessor.h>
#include <Processors/Transforms/ReverseTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Processors/Transforms/AddingSelectorTransform.h>
#include <Processors/Transforms/CopyTransform.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Merges/AggregatingSortedTransform.h>
#include <Processors/Merges/CollapsingSortedTransform.h>
#include <Processors/Merges/GraphiteRollupSortedTransform.h>
@ -16,17 +18,22 @@
#include <Processors/Merges/ReplacingSortedTransform.h>
#include <Processors/Merges/SummingSortedTransform.h>
#include <Processors/Merges/VersionedCollapsingTransform.h>
#include <Processors/QueryPlan/PartsSplitter.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Processors/Transforms/ReverseTransform.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/MergeTreeInOrderSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeThreadSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <Storages/VirtualColumnUtils.h>
#include <IO/Operators.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Common/logger_useful.h>
#include <base/sort.h>
#include <Poco/Logger.h>
#include <Common/JSONBuilder.h>
namespace ProfileEvents
@ -560,7 +567,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
static void addMergingFinal(
Pipe & pipe,
size_t num_output_streams,
const SortDescription & sort_description,
MergeTreeData::MergingParams merging_params,
Names partition_key_columns,
@ -607,56 +613,7 @@ static void addMergingFinal(
__builtin_unreachable();
};
if (num_output_streams <= 1 || sort_description.empty())
{
pipe.addTransform(get_merging_processor());
return;
}
ColumnNumbers key_columns;
key_columns.reserve(sort_description.size());
for (const auto & desc : sort_description)
key_columns.push_back(header.getPositionByName(desc.column_name));
pipe.addSimpleTransform([&](const Block & stream_header)
{
return std::make_shared<AddingSelectorTransform>(stream_header, num_output_streams, key_columns);
});
pipe.transform([&](OutputPortRawPtrs ports)
{
Processors transforms;
std::vector<OutputPorts::iterator> output_ports;
transforms.reserve(ports.size() + num_output_streams);
output_ports.reserve(ports.size());
for (auto & port : ports)
{
auto copier = std::make_shared<CopyTransform>(header, num_output_streams);
connect(*port, copier->getInputPort());
output_ports.emplace_back(copier->getOutputs().begin());
transforms.emplace_back(std::move(copier));
}
for (size_t i = 0; i < num_output_streams; ++i)
{
auto merge = get_merging_processor();
merge->setSelectorPosition(i);
auto input = merge->getInputs().begin();
/// Connect i-th merge with i-th input port of every copier.
for (size_t j = 0; j < ports.size(); ++j)
{
connect(*output_ports[j], *input);
++output_ports[j];
++input;
}
transforms.emplace_back(std::move(merge));
}
return transforms;
});
pipe.addTransform(get_merging_processor());
}
@ -710,8 +667,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index)
{
Pipe pipe;
Pipes pipes;
{
RangesInDataParts new_parts;
@ -738,21 +694,39 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
if (new_parts.empty())
continue;
pipe = read(std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder,
num_streams, 0, info.use_uncompressed_cache);
if (num_streams > 1 && metadata_for_reading->hasPrimaryKey())
{
// Let's split parts into layers to ensure data parallelism of final.
auto reading_step_getter = [this, &column_names, &info](auto parts)
{
return read(
std::move(parts),
column_names,
ReadFromMergeTree::ReadType::InOrder,
1 /* num_streams */,
0 /* min_marks_for_concurrent_read */,
info.use_uncompressed_cache);
};
pipes = buildPipesForReadingByPKRanges(
metadata_for_reading->getPrimaryKey(), std::move(new_parts), num_streams, context, std::move(reading_step_getter));
}
else
{
pipes.emplace_back(read(
std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder, num_streams, 0, info.use_uncompressed_cache));
}
/// Drop temporary columns, added by 'sorting_key_expr'
if (!out_projection)
out_projection = createProjection(pipe.getHeader());
out_projection = createProjection(pipes.front().getHeader());
}
auto sorting_expr = std::make_shared<ExpressionActions>(
metadata_for_reading->getSortingKey().expression->getActionsDAG().clone());
pipe.addSimpleTransform([sorting_expr](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, sorting_expr);
});
for (auto & pipe : pipes)
pipe.addSimpleTransform([sorting_expr](const Block & header)
{ return std::make_shared<ExpressionTransform>(header, sorting_expr); });
/// If do_not_merge_across_partitions_select_final is true and there is only one part in partition
/// with level > 0 then we won't postprocess this part
@ -760,7 +734,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 &&
parts_to_merge_ranges[range_index]->data_part->info.level > 0)
{
partition_pipes.emplace_back(std::move(pipe));
partition_pipes.emplace_back(Pipe::unitePipes(std::move(pipes)));
continue;
}
@ -777,21 +751,21 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
for (size_t i = 0; i < sort_columns_size; ++i)
sort_description.emplace_back(sort_columns[i], 1, 1);
addMergingFinal(
pipe,
std::min<size_t>(num_streams, settings.max_final_threads),
sort_description, data.merging_params, partition_key_columns, max_block_size);
for (auto & pipe : pipes)
addMergingFinal(
pipe,
sort_description,
data.merging_params,
partition_key_columns,
max_block_size);
partition_pipes.emplace_back(std::move(pipe));
partition_pipes.emplace_back(Pipe::unitePipes(std::move(pipes)));
}
if (!lonely_parts.empty())
{
RangesInDataParts new_parts;
size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size();
const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead(
settings.merge_tree_min_rows_for_concurrent_read,
settings.merge_tree_min_bytes_for_concurrent_read,

View File

@ -1,76 +0,0 @@
#include <Processors/Transforms/AddingSelectorTransform.h>
#include <Processors/Transforms/SelectorInfo.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
AddingSelectorTransform::AddingSelectorTransform(
const Block & header, size_t num_outputs_, ColumnNumbers key_columns_)
: ISimpleTransform(header, header, false)
, num_outputs(num_outputs_)
, key_columns(std::move(key_columns_))
, hash(0)
{
setInputNotNeededAfterRead(false);
if (num_outputs <= 1)
throw Exception("SplittingByHashTransform expects more than 1 outputs, got " + std::to_string(num_outputs),
ErrorCodes::LOGICAL_ERROR);
if (key_columns.empty())
throw Exception("SplittingByHashTransform cannot split by empty set of key columns",
ErrorCodes::LOGICAL_ERROR);
for (auto & column : key_columns)
if (column >= header.columns())
throw Exception("Invalid column number: " + std::to_string(column) +
". There is only " + std::to_string(header.columns()) + " columns in header",
ErrorCodes::LOGICAL_ERROR);
}
static void calculateWeakHash32(const Chunk & chunk, const ColumnNumbers & key_columns, WeakHash32 & hash)
{
auto num_rows = chunk.getNumRows();
const auto & columns = chunk.getColumns();
hash.reset(num_rows);
for (const auto & column_number : key_columns)
columns[column_number]->updateWeakHash32(hash);
}
static IColumn::Selector fillSelector(const WeakHash32 & hash, size_t num_outputs)
{
/// Row from interval [(2^32 / num_outputs) * i, (2^32 / num_outputs) * (i + 1)) goes to bucket with number i.
const auto & hash_data = hash.getData();
size_t num_rows = hash_data.size();
IColumn::Selector selector(num_rows);
for (size_t row = 0; row < num_rows; ++row)
{
selector[row] = hash_data[row]; /// [0, 2^32)
selector[row] *= num_outputs; /// [0, num_outputs * 2^32), selector stores 64 bit values.
selector[row] >>= 32u; /// [0, num_outputs)
}
return selector;
}
void AddingSelectorTransform::transform(Chunk & input_chunk, Chunk & output_chunk)
{
auto chunk_info = std::make_shared<SelectorInfo>();
calculateWeakHash32(input_chunk, key_columns, hash);
chunk_info->selector = fillSelector(hash, num_outputs);
input_chunk.swap(output_chunk);
output_chunk.setChunkInfo(std::move(chunk_info));
}
}

View File

@ -1,26 +0,0 @@
#pragma once
#include <Processors/IProcessor.h>
#include <Processors/ISimpleTransform.h>
#include <Core/ColumnNumbers.h>
#include <Common/WeakHash.h>
namespace DB
{
/// Add IColumn::Selector to chunk (see SelectorInfo.h).
/// Selector is filled by formula (WeakHash(key_columns) * num_outputs / MAX_INT).
class AddingSelectorTransform : public ISimpleTransform
{
public:
AddingSelectorTransform(const Block & header, size_t num_outputs_, ColumnNumbers key_columns_);
String getName() const override { return "AddingSelector"; }
void transform(Chunk & input_chunk, Chunk & output_chunk) override;
private:
size_t num_outputs;
ColumnNumbers key_columns;
WeakHash32 hash;
};
}

View File

@ -35,6 +35,8 @@ void CubeTransform::consume(Chunk chunk)
consumed_chunks.emplace_back(std::move(chunk));
}
MutableColumnPtr getColumnWithDefaults(Block const & header, size_t key, size_t n);
Chunk CubeTransform::generate()
{
if (!consumed_chunks.empty())
@ -53,8 +55,9 @@ Chunk CubeTransform::generate()
current_zero_columns.clear();
current_zero_columns.reserve(keys.size());
auto const & input_header = getInputPort().getHeader();
for (auto key : keys)
current_zero_columns.emplace_back(current_columns[key]->cloneEmpty()->cloneResized(num_rows));
current_zero_columns.emplace_back(getColumnWithDefaults(input_header, key, num_rows));
}
auto gen_chunk = std::move(cube_chunk);

View File

@ -0,0 +1,66 @@
#pragma once
#include <Interpreters/ExpressionActions.h>
#include <Processors/ISimpleTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/FilterTransform.h>
namespace DB
{
/// Could be used when the predicate given by expression_ is true only on one continuous range of values and input is monotonous by that value.
/// The following optimization applies: when a new chunk of data comes in we firstly execute the expression_ only on the first and the last row.
/// If it evaluates to true on both rows then the whole chunk is immediately passed to further steps.
/// Otherwise, we apply the expression_ to all rows.
class FilterSortedStreamByRange : public ISimpleTransform
{
public:
FilterSortedStreamByRange(
const Block & header_,
ExpressionActionsPtr expression_,
String filter_column_name_,
bool remove_filter_column_,
bool on_totals_ = false)
: ISimpleTransform(
header_,
FilterTransform::transformHeader(header_, expression_->getActionsDAG(), filter_column_name_, remove_filter_column_),
true)
, filter_transform(header_, expression_, filter_column_name_, remove_filter_column_, on_totals_)
{
}
String getName() const override { return "FilterSortedStreamByRange"; }
void transform(Chunk & chunk) override
{
int rows_before_filtration = chunk.getNumRows();
if (rows_before_filtration < 2)
{
filter_transform.transform(chunk);
return;
}
// Evaluate expression on just the first and the last row.
// If both of them satisfies conditions, than skip calculation for all the rows in between.
auto quick_check_columns = chunk.cloneEmptyColumns();
auto src_columns = chunk.detachColumns();
for (auto row : {0, rows_before_filtration - 1})
for (size_t col = 0; col < quick_check_columns.size(); ++col)
quick_check_columns[col]->insertFrom(*src_columns[col].get(), row);
chunk.setColumns(std::move(quick_check_columns), 2);
filter_transform.transform(chunk);
const bool all_rows_will_pass_filter = chunk.getNumRows() == 2;
chunk.setColumns(std::move(src_columns), rows_before_filtration);
// Not all rows satisfy conditions.
if (!all_rows_will_pass_filter)
filter_transform.transform(chunk);
}
private:
FilterTransform filter_transform;
};
}

View File

@ -32,7 +32,6 @@ public:
Status prepare() override;
protected:
void transform(Chunk & chunk) override;
private:

View File

@ -29,6 +29,14 @@ Chunk RollupTransform::merge(Chunks && chunks, bool final)
return Chunk(rollup_block.getColumns(), num_rows);
}
MutableColumnPtr getColumnWithDefaults(Block const & header, size_t key, size_t n)
{
auto const & col = header.getByPosition(key);
auto result_column = col.column->cloneEmpty();
col.type->insertManyDefaultsInto(*result_column, n);
return result_column;
}
Chunk RollupTransform::generate()
{
if (!consumed_chunks.empty())
@ -51,7 +59,7 @@ Chunk RollupTransform::generate()
auto num_rows = gen_chunk.getNumRows();
auto columns = gen_chunk.getColumns();
columns[key] = columns[key]->cloneEmpty()->cloneResized(num_rows);
columns[key] = getColumnWithDefaults(getInputPort().getHeader(), key, num_rows);
Chunks chunks;
chunks.emplace_back(std::move(columns), num_rows);

View File

@ -1,14 +0,0 @@
#pragma once
#include <Processors/Chunk.h>
#include <Common/PODArray.h>
namespace DB
{
/// ChunkInfo with IColumn::Selector. It is added by AddingSelectorTransform.
struct SelectorInfo : public ChunkInfo
{
IColumn::Selector selector;
};
}

View File

@ -28,7 +28,10 @@ void printPipeline(const Processors & processors, const Statuses & statuses, Wri
/// Nodes // TODO quoting and escaping
for (const auto & processor : processors)
{
out << " n" << get_proc_id(*processor) << "[label=\"" << processor->getName() << processor->getDescription();
auto description = processor->getDescription();
if (!description.empty())
description = ": " + description;
out << " n" << get_proc_id(*processor) << "[label=\"" << processor->getName() << description;
if (statuses_iter != statuses.end())
{

View File

@ -7,6 +7,7 @@
#include <Storages/Cache/ExternalDataSourceCache.h>
#include <Storages/Cache/RemoteFileMetadataFactory.h>
#include <base/errnoToString.h>
#include <base/sort.h>
#include <Common/logger_useful.h>
#include <base/sleep.h>
#include <Poco/Logger.h>
@ -229,7 +230,7 @@ void ExternalDataSourceCache::initOnce(ContextPtr context, const String & root_d
LOG_INFO(
log, "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", root_dir_, limit_size_);
splitInto<','>(root_dirs, root_dir_);
std::sort(root_dirs.begin(), root_dirs.end());
::sort(root_dirs.begin(), root_dirs.end());
local_cache_bytes_read_before_flush = bytes_read_before_flush_;
lru_caches = std::make_unique<RemoteFileCacheType>(limit_size_);

201
src/Storages/Freeze.cpp Normal file
View File

@ -0,0 +1,201 @@
#include <Storages/Freeze.h>
#include <Disks/ObjectStorages/IMetadataStorage.h>
#include <Storages/PartitionCommands.h>
#include <Common/escapeForFileName.h>
#include <Common/logger_useful.h>
namespace DB
{
void FreezeMetaData::fill(const StorageReplicatedMergeTree & storage)
{
is_replicated = storage.supportsReplication();
is_remote = storage.isRemote();
replica_name = storage.getReplicaName();
zookeeper_name = storage.getZooKeeperName();
table_shared_id = storage.getTableSharedID();
}
void FreezeMetaData::save(DiskPtr data_disk, const String & path) const
{
auto metadata_storage = data_disk->getMetadataStorage();
auto file_path = getFileName(path);
auto tx = metadata_storage->createTransaction();
WriteBufferFromOwnString buffer;
writeIntText(version, buffer);
buffer.write("\n", 1);
writeBoolText(is_replicated, buffer);
buffer.write("\n", 1);
writeBoolText(is_remote, buffer);
buffer.write("\n", 1);
writeString(replica_name, buffer);
buffer.write("\n", 1);
writeString(zookeeper_name, buffer);
buffer.write("\n", 1);
writeString(table_shared_id, buffer);
buffer.write("\n", 1);
tx->writeStringToFile(file_path, buffer.str());
tx->commit();
}
bool FreezeMetaData::load(DiskPtr data_disk, const String & path)
{
auto metadata_storage = data_disk->getMetadataStorage();
auto file_path = getFileName(path);
if (!metadata_storage->exists(file_path))
return false;
auto metadata_str = metadata_storage->readFileToString(file_path);
ReadBufferFromString buffer(metadata_str);
readIntText(version, buffer);
if (version != 1)
{
LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown freezed metadata version: {}", version);
return false;
}
DB::assertChar('\n', buffer);
readBoolText(is_replicated, buffer);
DB::assertChar('\n', buffer);
readBoolText(is_remote, buffer);
DB::assertChar('\n', buffer);
readString(replica_name, buffer);
DB::assertChar('\n', buffer);
readString(zookeeper_name, buffer);
DB::assertChar('\n', buffer);
readString(table_shared_id, buffer);
DB::assertChar('\n', buffer);
return true;
}
void FreezeMetaData::clean(DiskPtr data_disk, const String & path)
{
auto metadata_storage = data_disk->getMetadataStorage();
auto fname = getFileName(path);
if (metadata_storage->exists(fname))
{
auto tx = metadata_storage->createTransaction();
tx->unlinkFile(fname);
tx->commit();
}
}
String FreezeMetaData::getFileName(const String & path)
{
return fs::path(path) / "frozen_metadata.txt";
}
BlockIO Unfreezer::unfreeze(const String & backup_name, ContextPtr local_context)
{
LOG_DEBUG(log, "Unfreezing backup {}", backup_name);
auto disks_map = local_context->getDisksMap();
Disks disks;
for (auto & [name, disk]: disks_map)
{
disks.push_back(disk);
}
auto backup_path = fs::path(backup_directory_prefix) / escapeForFileName(backup_name);
auto store_path = backup_path / "store";
PartitionCommandsResultInfo result_info;
for (const auto & disk: disks)
{
if (!disk->exists(store_path))
continue;
for (auto prefix_it = disk->iterateDirectory(store_path); prefix_it->isValid(); prefix_it->next())
{
auto prefix_directory = store_path / prefix_it->name();
for (auto table_it = disk->iterateDirectory(prefix_directory); table_it->isValid(); table_it->next())
{
auto table_directory = prefix_directory / table_it->name();
auto current_result_info = unfreezePartitionsFromTableDirectory([] (const String &) { return true; }, backup_name, {disk}, table_directory, local_context);
for (auto & command_result : current_result_info)
{
command_result.command_type = "SYSTEM UNFREEZE";
}
result_info.insert(
result_info.end(),
std::make_move_iterator(current_result_info.begin()),
std::make_move_iterator(current_result_info.end()));
}
}
if (disk->exists(backup_path))
{
disk->removeRecursive(backup_path);
}
}
BlockIO result;
if (!result_info.empty())
{
result.pipeline = QueryPipeline(convertCommandsResultToSource(result_info));
}
return result;
}
bool Unfreezer::removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context)
{
if (disk->supportZeroCopyReplication())
{
FreezeMetaData meta;
if (meta.load(disk, path))
{
if (meta.is_replicated)
{
FreezeMetaData::clean(disk, path);
return StorageReplicatedMergeTree::removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "", local_context);
}
}
}
disk->removeRecursive(path);
return false;
}
PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory, ContextPtr local_context)
{
PartitionCommandsResultInfo result;
for (const auto & disk : disks)
{
if (!disk->exists(table_directory))
continue;
for (auto it = disk->iterateDirectory(table_directory); it->isValid(); it->next())
{
const auto & partition_directory = it->name();
/// Partition ID is prefix of part directory name: <partition id>_<rest of part directory name>
auto found = partition_directory.find('_');
if (found == std::string::npos)
continue;
auto partition_id = partition_directory.substr(0, found);
if (!matcher(partition_id))
continue;
const auto & path = it->path();
bool keep_shared = removeFreezedPart(disk, path, partition_directory, local_context);
result.push_back(PartitionCommandResultInfo{
.partition_id = partition_id,
.part_name = partition_directory,
.backup_path = disk->getPath() + table_directory.generic_string(),
.part_backup_path = disk->getPath() + path,
.backup_name = backup_name,
});
LOG_DEBUG(log, "Unfreezed part by path {}, keep shared data: {}", disk->getPath() + path, keep_shared);
}
}
LOG_DEBUG(log, "Unfreezed {} parts", result.size());
return result;
}
}

45
src/Storages/Freeze.h Normal file
View File

@ -0,0 +1,45 @@
#pragma once
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeData.h>
namespace DB
{
/// Special metadata used during freeze table. Required for zero-copy
/// replication.
struct FreezeMetaData
{
public:
void fill(const StorageReplicatedMergeTree & storage);
void save(DiskPtr data_disk, const String & path) const;
bool load(DiskPtr data_disk, const String & path);
static void clean(DiskPtr data_disk, const String & path);
private:
static String getFileName(const String & path);
public:
int version = 1;
bool is_replicated{false};
bool is_remote{false};
String replica_name;
String zookeeper_name;
String table_shared_id;
};
class Unfreezer
{
public:
PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory, ContextPtr local_context);
BlockIO unfreeze(const String & backup_name, ContextPtr local_context);
private:
Poco::Logger * log = &Poco::Logger::get("Unfreezer");
static constexpr std::string_view backup_directory_prefix = "shadow";
static bool removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context);
};
}

View File

@ -56,6 +56,7 @@
#include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/VirtualColumnUtils.h>
#include <Storages/Freeze.h>
#include <Common/Increment.h>
#include <Common/SimpleIncrement.h>
#include <Common/Stopwatch.h>
@ -1846,6 +1847,62 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
}
}
size_t MergeTreeData::clearOldBrokenPartsFromDetachedDirecory()
{
/**
* Remove old (configured by setting) broken detached parts.
* Only parts with certain prefixes are removed. These prefixes
* are such that it is guaranteed that they will never be needed
* and need to be cleared. ctime is used to check when file was
* moved to detached/ directory (see https://unix.stackexchange.com/a/211134)
*/
DetachedPartsInfo detached_parts = getDetachedParts();
if (detached_parts.empty())
return 0;
PartsTemporaryRename renamed_parts(*this, "detached/");
for (const auto & part_info : detached_parts)
{
if (!part_info.valid_name || part_info.prefix.empty())
continue;
const auto & removable_detached_parts_prefixes = DetachedPartInfo::DETACHED_REASONS_REMOVABLE_BY_TIMEOUT;
bool can_be_removed_by_timeout = std::find(
removable_detached_parts_prefixes.begin(),
removable_detached_parts_prefixes.end(),
part_info.prefix) != removable_detached_parts_prefixes.end();
if (!can_be_removed_by_timeout)
continue;
time_t current_time = time(nullptr);
ssize_t threshold = current_time - getSettings()->merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds;
auto path = fs::path(relative_data_path) / "detached" / part_info.dir_name;
time_t last_change_time = part_info.disk->getLastChanged(path);
time_t last_modification_time = part_info.disk->getLastModified(path).epochTime();
time_t last_touch_time = std::max(last_change_time, last_modification_time);
if (last_touch_time == 0 || last_touch_time >= threshold)
continue;
renamed_parts.addPart(part_info.dir_name, "deleting_" + part_info.dir_name, part_info.disk);
}
LOG_INFO(log, "Will clean up {} detached parts", renamed_parts.old_and_new_names.size());
renamed_parts.tryRenameAll();
for (const auto & [old_name, new_name, disk] : renamed_parts.old_and_new_names)
{
removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name, false);
LOG_DEBUG(log, "Removed broken detached part {} due to a timeout for broken detached parts", old_name);
}
return renamed_parts.old_and_new_names.size();
}
size_t MergeTreeData::clearOldWriteAheadLogs()
{
DataPartsVector parts = getDataPartsVectorForInternalUsage();
@ -1892,6 +1949,7 @@ size_t MergeTreeData::clearOldWriteAheadLogs()
auto disk_ptr = *disk_it;
if (disk_ptr->isBroken())
continue;
for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next())
{
auto min_max_block_number = MergeTreeWriteAheadLog::tryParseMinMaxBlockNumber(it->name());
@ -6112,51 +6170,15 @@ bool MergeTreeData::removeDetachedPart(DiskPtr disk, const String & path, const
return false;
}
PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn matcher, const String & backup_name, ContextPtr)
PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn matcher, const String & backup_name, ContextPtr local_context)
{
auto backup_path = fs::path("shadow") / escapeForFileName(backup_name) / relative_data_path;
LOG_DEBUG(log, "Unfreezing parts by path {}", backup_path.generic_string());
PartitionCommandsResultInfo result;
auto disks = getStoragePolicy()->getDisks();
for (const auto & disk : getStoragePolicy()->getDisks())
{
if (!disk->exists(backup_path))
continue;
for (auto it = disk->iterateDirectory(backup_path); it->isValid(); it->next())
{
const auto & partition_directory = it->name();
/// Partition ID is prefix of part directory name: <partition id>_<rest of part directory name>
auto found = partition_directory.find('_');
if (found == std::string::npos)
continue;
auto partition_id = partition_directory.substr(0, found);
if (!matcher(partition_id))
continue;
const auto & path = it->path();
bool keep_shared = removeDetachedPart(disk, path, partition_directory, true);
result.push_back(PartitionCommandResultInfo{
.partition_id = partition_id,
.part_name = partition_directory,
.backup_path = disk->getPath() + backup_path.generic_string(),
.part_backup_path = disk->getPath() + path,
.backup_name = backup_name,
});
LOG_DEBUG(log, "Unfreezed part by path {}, keep shared data: {}", disk->getPath() + path, keep_shared);
}
}
LOG_DEBUG(log, "Unfreezed {} parts", result.size());
return result;
return Unfreezer().unfreezePartitionsFromTableDirectory(matcher, backup_name, disks, backup_path, local_context);
}
bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const

View File

@ -622,6 +622,8 @@ public:
/// Delete WAL files containing parts, that all already stored on disk.
size_t clearOldWriteAheadLogs();
size_t clearOldBrokenPartsFromDetachedDirecory();
/// Delete all directories which names begin with "tmp"
/// Must be called with locked lockForShare() because it's using relative_data_path.
size_t clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes = {"tmp_", });
@ -988,6 +990,9 @@ public:
/// Mutex for currently_submerging_parts and currently_emerging_parts
mutable std::mutex currently_submerging_emerging_mutex;
/// Used for freezePartitionsByMatcher and unfreezePartitionsByMatcher
using MatcherFn = std::function<bool(const String &)>;
protected:
friend class IMergeTreeDataPart;
friend class MergeTreeDataMergerMutator;
@ -1178,7 +1183,6 @@ protected:
bool isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node, const StorageMetadataPtr & metadata_snapshot) const;
/// Common part for |freezePartition()| and |freezeAll()|.
using MatcherFn = std::function<bool(const String &)>;
PartitionCommandsResultInfo freezePartitionsByMatcher(MatcherFn matcher, const StorageMetadataPtr & metadata_snapshot, const String & with_name, ContextPtr context);
PartitionCommandsResultInfo unfreezePartitionsByMatcher(MatcherFn matcher, const String & backup_name, ContextPtr context);

View File

@ -156,6 +156,16 @@ struct DetachedPartInfo : public MergeTreePartInfo
"covered-by-broken",
});
static constexpr auto DETACHED_REASONS_REMOVABLE_BY_TIMEOUT = std::to_array<std::string_view>({
"broken",
"unexpected",
"noquorum",
"ignored",
"broken-on-start",
"deleting",
"clone"
});
/// NOTE: It may parse part info incorrectly.
/// For example, if prefix contains '_' or if DETACH_REASONS doesn't contain prefix.
// This method has different semantics with MergeTreePartInfo::tryParsePartName.

View File

@ -61,6 +61,8 @@ struct Settings;
M(UInt64, merge_selecting_sleep_ms, 5000, "Sleep time for merge selecting when no part selected, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \
M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \
M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \
M(UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30, "Remove old broken detached parts in the background if they remained intouched for a specified by this setting period of time.", 0) \
M(UInt64, merge_tree_enable_clear_old_broken_detached, false, "Enable clearing old broken detached parts operation in background.", 0) \
M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \
\
/** Inserts settings. */ \

View File

@ -65,6 +65,8 @@ void ReplicatedMergeTreeCleanupThread::iterate()
/// do it under share lock
storage.clearOldWriteAheadLogs();
storage.clearOldTemporaryDirectories(storage.getSettings()->temporary_directories_lifetime.totalSeconds());
if (storage.getSettings()->merge_tree_enable_clear_old_broken_detached)
storage.clearOldBrokenPartsFromDetachedDirecory();
}
/// This is loose condition: no problem if we actually had lost leadership at this moment

View File

@ -1184,6 +1184,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign
}, common_assignee_trigger, getStorageID()), /* need_trigger */ false);
scheduled = true;
}
if (auto lock = time_after_previous_cleanup_parts.compareAndRestartDeferred(
getSettings()->merge_tree_clear_old_parts_interval_seconds))
{
@ -1197,6 +1198,8 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign
cleared_count += clearOldWriteAheadLogs();
cleared_count += clearOldMutations();
cleared_count += clearEmptyParts();
if (getSettings()->merge_tree_enable_clear_old_broken_detached)
cleared_count += clearOldBrokenPartsFromDetachedDirecory();
return cleared_count;
/// TODO maybe take into account number of cleared objects when calculating backoff
}, common_assignee_trigger, getStorageID()), /* need_trigger */ false);

View File

@ -130,6 +130,8 @@ private:
AtomicStopwatch time_after_previous_cleanup_parts;
/// For clearOldTemporaryDirectories.
AtomicStopwatch time_after_previous_cleanup_temporary_directories;
/// For clearOldBrokenDetachedParts
AtomicStopwatch time_after_previous_cleanup_broken_detached_parts;
/// Mutex for parts currently processing in background
/// merging (also with TTL), mutating or moving.

View File

@ -36,6 +36,7 @@
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
#include <Storages/MergeTree/LeaderElection.h>
#include <Storages/MergeTree/ZeroCopyLock.h>
#include <Storages/Freeze.h>
#include <Databases/DatabaseOnDisk.h>
@ -466,6 +467,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
/// don't allow to reinitialize them, delete each of them immediately.
clearOldTemporaryDirectories(0, {"tmp_", "delete_tmp_"});
clearOldWriteAheadLogs();
if (getSettings()->merge_tree_enable_clear_old_broken_detached)
clearOldBrokenPartsFromDetachedDirecory();
}
createNewZooKeeperNodes();
@ -8155,107 +8158,6 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode(
}
}
namespace
{
/// Special metadata used during freeze table. Required for zero-copy
/// replication.
struct FreezeMetaData
{
public:
void fill(const StorageReplicatedMergeTree & storage)
{
is_replicated = storage.supportsReplication();
is_remote = storage.isRemote();
replica_name = storage.getReplicaName();
zookeeper_name = storage.getZooKeeperName();
table_shared_id = storage.getTableSharedID();
}
void save(DiskPtr data_disk, const String & path) const
{
auto metadata_storage = data_disk->getMetadataStorage();
auto file_path = getFileName(path);
auto tx = metadata_storage->createTransaction();
WriteBufferFromOwnString buffer;
writeIntText(version, buffer);
buffer.write("\n", 1);
writeBoolText(is_replicated, buffer);
buffer.write("\n", 1);
writeBoolText(is_remote, buffer);
buffer.write("\n", 1);
writeString(replica_name, buffer);
buffer.write("\n", 1);
writeString(zookeeper_name, buffer);
buffer.write("\n", 1);
writeString(table_shared_id, buffer);
buffer.write("\n", 1);
tx->writeStringToFile(file_path, buffer.str());
tx->commit();
}
bool load(DiskPtr data_disk, const String & path)
{
auto metadata_storage = data_disk->getMetadataStorage();
auto file_path = getFileName(path);
if (!metadata_storage->exists(file_path))
return false;
auto metadata_str = metadata_storage->readFileToString(file_path);
ReadBufferFromString buffer(metadata_str);
readIntText(version, buffer);
if (version != 1)
{
LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown freezed metadata version: {}", version);
return false;
}
DB::assertChar('\n', buffer);
readBoolText(is_replicated, buffer);
DB::assertChar('\n', buffer);
readBoolText(is_remote, buffer);
DB::assertChar('\n', buffer);
readString(replica_name, buffer);
DB::assertChar('\n', buffer);
readString(zookeeper_name, buffer);
DB::assertChar('\n', buffer);
readString(table_shared_id, buffer);
DB::assertChar('\n', buffer);
return true;
}
static void clean(DiskPtr data_disk, const String & path)
{
auto metadata_storage = data_disk->getMetadataStorage();
auto fname = getFileName(path);
if (metadata_storage->exists(fname))
{
auto tx = metadata_storage->createTransaction();
tx->unlinkFile(fname);
tx->commit();
}
}
private:
static String getFileName(const String & path)
{
return fs::path(path) / "frozen_metadata.txt";
}
public:
int version = 1;
bool is_replicated;
bool is_remote;
String replica_name;
String zookeeper_name;
String table_shared_id;
};
}
bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed)
{
if (disk->supportZeroCopyReplication())
@ -8266,14 +8168,14 @@ bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String &
if (meta.load(disk, path))
{
FreezeMetaData::clean(disk, path);
return removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "");
return removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "", getContext());
}
}
else
{
String table_id = getTableSharedID();
return removeSharedDetachedPart(disk, path, part_name, table_id, zookeeper_name, replica_name, zookeeper_path);
return removeSharedDetachedPart(disk, path, part_name, table_id, zookeeper_name, replica_name, zookeeper_path, getContext());
}
}
@ -8284,11 +8186,11 @@ bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String &
bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid,
const String &, const String & detached_replica_name, const String & detached_zookeeper_path)
const String &, const String & detached_replica_name, const String & detached_zookeeper_path, ContextPtr local_context)
{
bool keep_shared = false;
zkutil::ZooKeeperPtr zookeeper = getZooKeeper();
zkutil::ZooKeeperPtr zookeeper = local_context->getZooKeeper();
NameSet files_not_to_remove;
fs::path checksums = fs::path(path) / IMergeTreeDataPart::FILE_FOR_REFERENCES_CHECK;
@ -8299,7 +8201,7 @@ bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const St
String id = disk->getUniqueId(checksums);
bool can_remove = false;
std::tie(can_remove, files_not_to_remove) = StorageReplicatedMergeTree::unlockSharedDataByID(id, table_uuid, part_name,
detached_replica_name, disk, zookeeper, getContext()->getReplicatedMergeTreeSettings(), log,
detached_replica_name, disk, zookeeper, local_context->getReplicatedMergeTreeSettings(), &Poco::Logger::get("StorageReplicatedMergeTree"),
detached_zookeeper_path);
keep_shared = !can_remove;

View File

@ -300,6 +300,9 @@ public:
/// Check if there are new broken disks and enqueue part recovery tasks.
void checkBrokenDisks();
static bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid,
const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, ContextPtr local_context);
private:
std::atomic_bool are_restoring_replica {false};
@ -792,9 +795,6 @@ private:
bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed) override;
bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid,
const String & zookeeper_name, const String & replica_name, const String & zookeeper_path);
/// Create freeze metadata for table and save in zookeeper. Required only if zero-copy replication enabled.
void createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr part, String backup_part_path) const override;

View File

@ -93,7 +93,7 @@ def get_failed_report(
elapsed_seconds=0,
with_coverage=False,
)
return [build_result], [[""]], [""]
return [build_result], [[""]], [GITHUB_RUN_URL]
def process_report(

View File

@ -187,7 +187,11 @@ def gen_tags(version: ClickHouseVersion, release_type: str) -> List[str]:
def buildx_args(bucket_prefix: str, arch: str) -> List[str]:
args = [f"--platform=linux/{arch}", f"--label=build-url={GITHUB_RUN_URL}"]
args = [
f"--platform=linux/{arch}",
f"--label=build-url={GITHUB_RUN_URL}",
f"--label=com.clickhouse.build.githash={git.sha}",
]
if bucket_prefix:
url = p.join(bucket_prefix, BUCKETS[arch]) # to prevent a double //
args.append(f"--build-arg=REPOSITORY='{url}'")

View File

@ -27,3 +27,9 @@ def corrupt_part_data_by_path(node, part_path):
],
privileged=True,
)
def break_part(node, table, part_name):
path = f"/var/lib/clickhouse/data/default/{table}/{part_name}/columns.txt"
print(f"Corrupting part {part_name}, removing file: {path}")
node.exec_in_container(["bash", "-c", f"rm {path}"])

View File

@ -0,0 +1,149 @@
import pytest
from helpers.cluster import ClickHouseCluster
from multiprocessing.dummy import Pool
from helpers.corrupt_part_data_on_disk import corrupt_part_data_on_disk
from helpers.corrupt_part_data_on_disk import break_part
import time
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance("node1", stay_alive=True, with_zookeeper=True)
path_to_data = "/var/lib/clickhouse/"
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def remove_broken_detached_part_impl(table, node, expect_broken_prefix):
assert (
node.query(
f"SELECT COUNT() FROM system.parts WHERE table='{table}' AND active=1"
)
== "4\n"
)
path_to_detached = path_to_data + f"data/default/{table}/detached/"
result = node.exec_in_container(["ls", path_to_detached])
assert result.strip() == ""
corrupt_part_data_on_disk(node, table, "all_3_3_0")
break_part(node, table, "all_3_3_0")
node.query(f"ALTER TABLE {table} DETACH PART 'all_1_1_0'")
result = node.exec_in_container(["touch", f"{path_to_detached}trash"])
node.exec_in_container(["mkdir", f"{path_to_detached}../broken_all_fake"])
node.exec_in_container(
["touch", "-t", "1312031429.30", f"{path_to_detached}../broken_all_fake"]
)
result = node.exec_in_container(["stat", f"{path_to_detached}../broken_all_fake"])
print(result)
assert "Modify: 2013-12-03" in result
node.exec_in_container(
[
"mv",
f"{path_to_detached}../broken_all_fake",
f"{path_to_detached}broken_all_fake",
]
)
node.exec_in_container(["mkdir", f"{path_to_detached}../unexpected_all_42_1337_5"])
node.exec_in_container(
[
"touch",
"-t",
"1312031429.30",
f"{path_to_detached}../unexpected_all_42_1337_5",
]
)
result = node.exec_in_container(
["stat", f"{path_to_detached}../unexpected_all_42_1337_5"]
)
print(result)
assert "Modify: 2013-12-03" in result
node.exec_in_container(
[
"mv",
f"{path_to_detached}../unexpected_all_42_1337_5",
f"{path_to_detached}unexpected_all_42_1337_5",
]
)
result = node.query(
f"CHECK TABLE {table}", settings={"check_query_single_value_result": 0}
)
assert "all_3_3_0\t0" in result
node.query(f"DETACH TABLE {table}")
node.query(f"ATTACH TABLE {table}")
result = node.exec_in_container(["ls", path_to_detached])
print(result)
assert f"{expect_broken_prefix}_all_3_3_0" in result
assert "all_1_1_0" in result
assert "trash" in result
assert "broken_all_fake" in result
assert "unexpected_all_42_1337_5" in result
time.sleep(15)
result = node.exec_in_container(["ls", path_to_detached])
print(result)
assert f"{expect_broken_prefix}_all_3_3_0" not in result
assert "all_1_1_0" in result
assert "trash" in result
assert "broken_all_fake" in result
assert "unexpected_all_42_1337_5" not in result
node.query(f"DROP TABLE {table} SYNC")
def test_remove_broken_detached_part_merge_tree(started_cluster):
node1.query(
"""
CREATE TABLE
mt(id UInt32, value Int32)
ENGINE = MergeTree() ORDER BY id
SETTINGS
merge_tree_enable_clear_old_broken_detached=1,
merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds=5;
"""
)
for i in range(4):
node1.query(
f"INSERT INTO mt SELECT number, number * number FROM numbers ({i * 100000}, 100000)"
)
remove_broken_detached_part_impl("mt", node1, "broken-on-start")
def test_remove_broken_detached_part_replicated_merge_tree(started_cluster):
node1.query(
f"""
CREATE TABLE
replicated_mt(date Date, id UInt32, value Int32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{node1.name}') ORDER BY id
SETTINGS
merge_tree_enable_clear_old_broken_detached=1,
merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds=5,
cleanup_delay_period=1,
cleanup_delay_period_random_add=0;
"""
)
for i in range(4):
node1.query(
f"INSERT INTO replicated_mt SELECT toDate('2019-10-01'), number, number * number FROM numbers ({i * 100000}, 100000)"
)
remove_broken_detached_part_impl("replicated_mt", node1, "broken")

View File

@ -525,6 +525,38 @@ def test_freeze_unfreeze(cluster, node_name):
)
@pytest.mark.parametrize("node_name", ["node"])
def test_freeze_system_unfreeze(cluster, node_name):
node = cluster.instances[node_name]
create_table(node, "s3_test")
create_table(node, "s3_test_removed")
minio = cluster.minio_client
node.query(
"INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-04", 4096))
)
node.query(
"INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-04", 4096))
)
node.query("ALTER TABLE s3_test FREEZE WITH NAME 'backup3'")
node.query("ALTER TABLE s3_test_removed FREEZE WITH NAME 'backup3'")
node.query("TRUNCATE TABLE s3_test")
node.query("DROP TABLE s3_test_removed NO DELAY")
assert (
len(list(minio.list_objects(cluster.minio_bucket, "data/")))
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
)
# Unfreeze all data from backup3.
node.query("SYSTEM UNFREEZE WITH NAME 'backup3'")
# Data should be removed from S3.
assert (
len(list(minio.list_objects(cluster.minio_bucket, "data/"))) == FILES_OVERHEAD
)
@pytest.mark.parametrize("node_name", ["node"])
def test_s3_disk_apply_new_settings(cluster, node_name):
node = cluster.instances[node_name]

View File

@ -412,7 +412,7 @@ def wait_mutations(node, table, seconds):
assert mutations == "0\n"
def test_s3_zero_copy_unfreeze(cluster):
def s3_zero_copy_unfreeze_base(cluster, unfreeze_query_template):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
@ -455,12 +455,12 @@ def test_s3_zero_copy_unfreeze(cluster):
check_objects_exisis(cluster, objects11)
node1.query("ALTER TABLE unfreeze_test UNFREEZE WITH NAME 'freeze_backup1'")
node1.query(f"{unfreeze_query_template} 'freeze_backup1'")
wait_mutations(node1, "unfreeze_test", 10)
check_objects_exisis(cluster, objects12)
node2.query("ALTER TABLE unfreeze_test UNFREEZE WITH NAME 'freeze_backup2'")
node2.query(f"{unfreeze_query_template} 'freeze_backup2'")
wait_mutations(node2, "unfreeze_test", 10)
check_objects_not_exisis(cluster, objects12)
@ -469,7 +469,15 @@ def test_s3_zero_copy_unfreeze(cluster):
node2.query("DROP TABLE IF EXISTS unfreeze_test NO DELAY")
def test_s3_zero_copy_drop_detached(cluster):
def test_s3_zero_copy_unfreeze_alter(cluster):
s3_zero_copy_unfreeze_base(cluster, "ALTER TABLE unfreeze_test UNFREEZE WITH NAME")
def test_s3_zero_copy_unfreeze_system(cluster):
s3_zero_copy_unfreeze_base(cluster, "SYSTEM UNFREEZE WITH NAME")
def s3_zero_copy_drop_detached(cluster, unfreeze_query_template):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
@ -498,8 +506,8 @@ def test_s3_zero_copy_drop_detached(cluster):
objects_diff = list(set(objects2) - set(objects1))
node1.query("ALTER TABLE drop_detached_test UNFREEZE WITH NAME 'detach_backup2'")
node1.query("ALTER TABLE drop_detached_test UNFREEZE WITH NAME 'detach_backup1'")
node1.query(f"{unfreeze_query_template} 'detach_backup2'")
node1.query(f"{unfreeze_query_template} 'detach_backup1'")
node1.query("ALTER TABLE drop_detached_test DETACH PARTITION '0'")
node1.query("ALTER TABLE drop_detached_test DETACH PARTITION '1'")
@ -554,6 +562,16 @@ def test_s3_zero_copy_drop_detached(cluster):
check_objects_not_exisis(cluster, objects1)
def test_s3_zero_copy_drop_detached_alter(cluster):
s3_zero_copy_drop_detached(
cluster, "ALTER TABLE drop_detached_test UNFREEZE WITH NAME"
)
def test_s3_zero_copy_drop_detached_system(cluster):
s3_zero_copy_drop_detached(cluster, "SYSTEM UNFREEZE WITH NAME")
def test_s3_zero_copy_concurrent_merge(cluster):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]

View File

@ -12,6 +12,7 @@
<value>ZSTD</value>
<value>DoubleDelta</value>
<value>Gorilla</value>
<value>FPC</value>
</values>
</substitution>
<substitution>

View File

@ -12,6 +12,7 @@
<value>ZSTD</value>
<value>DoubleDelta</value>
<value>Gorilla</value>
<value>FPC</value>
</values>
</substitution>
<substitution>

View File

@ -3,7 +3,6 @@
<substitution>
<name>dictionary_layout</name>
<values>
<value>flat</value>
<value>hashed</value>
<value>hashed_array</value>
</values>
@ -23,7 +22,7 @@
(
id UInt64,
parent_id UInt64
) ENGINE = Memory;
) ENGINE = MergeTree ORDER BY id;
</create_query>
<create_query>
@ -34,22 +33,43 @@
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(DB 'default' TABLE 'hierarchical_dictionary_source_table'))
LAYOUT({dictionary_layout}())
LAYOUT({dictionary_layout})
LIFETIME(0);
</create_query>
<create_query>
CREATE DICTIONARY hierarchical_flat_dictionary
(
id UInt64,
parent_id UInt64 HIERARCHICAL
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(DB 'default' TABLE 'hierarchical_dictionary_source_table'))
LAYOUT(FLAT(max_array_size 1000001))
LIFETIME(0);
</create_query>
<fill_query>
INSERT INTO hierarchical_dictionary_source_table
SELECT number, rand64() % 250000
WITH 5000 AS first_level_start, 50000 as second_level_start
SELECT
(number + 1) as id,
multiIf(id > second_level_start, id % (second_level_start - first_level_start) + first_level_start + 1,
id > first_level_start, (id % first_level_start) + 1,
0) as parent_id
FROM system.numbers
LIMIT 500000;
LIMIT 1000000;
</fill_query>
<query>
SELECT {func}('hierarchical_{dictionary_layout}_dictionary', id) FROM hierarchical_{dictionary_layout}_dictionary FORMAT Null;
SELECT {func}('hierarchical_flat_dictionary', number + 1) FROM numbers(1000000) FORMAT Null;
</query>
<query>
SELECT {func}('hierarchical_{dictionary_layout}_dictionary', number + 1) FROM numbers(1000000) FORMAT Null;
</query>
<drop_query>DROP TABLE IF EXISTS hierarchical_dictionary_source_table;</drop_query>
<drop_query>DROP DICTIONARY IF EXISTS hierarchical_{dictionary_layout}_dictionary;</drop_query>
<drop_query>DROP DICTIONARY IF EXISTS hierarchical_flat_dictionary;</drop_query>
</test>

View File

@ -18,6 +18,7 @@
<value>collapsing_final_16p_str_keys_rnd</value>
<value>collapsing_final_1024p_ord</value>
<value>collapsing_final_1024p_rnd</value>
<value>collapsing_final_1p_ord</value>
</values>
</substitution>
</substitutions>
@ -30,6 +31,7 @@
<create_query>create table collapsing_final_16p_str_keys_rnd (key1 UInt32, key2 String, key3 String, key4 String, key5 String, key6 String, key7 String, key8 String, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1, key2, key3, key4, key5, key6, key7, key8) partition by key1 % 16 </create_query>
<create_query>create table collapsing_final_1024p_ord (key1 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1) partition by intDiv(key1, 8192 * 2) </create_query>
<create_query>create table collapsing_final_1024p_rnd (key1 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1) partition by key1 % 1024 </create_query>
<create_query>create table collapsing_final_1p_ord (key1 UInt64, key2 UInt64, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1, key2)</create_query>
<!-- 16 parts, 8192 * 1024 rows each -->
<fill_query>insert into collapsing_final_16p_ord select number, number, 1, number from numbers_mt(8388608) </fill_query>
@ -43,6 +45,9 @@
<fill_query>insert into collapsing_final_1024p_ord select number, 1, number from numbers_mt(16777216) </fill_query>
<fill_query>insert into collapsing_final_1024p_rnd select number, 1, number from numbers_mt(16777216) </fill_query>
<!-- 1 big part of 5e7 rows -->
<fill_query>insert into collapsing_final_1p_ord select number, number + 1, 1, number from numbers_mt(5e7)</fill_query>
<fill_query>optimize table {collapsing} final</fill_query>
<query>SELECT count() FROM {collapsing} final</query>

View File

@ -0,0 +1,31 @@
<test>
<substitutions>
<substitution>
<name>func</name>
<values>
<value>not</value>
</values>
</substitution>
<substitution>
<name>expr</name>
<values>
<value>number</value>
<value>toUInt32(number)</value>
<value>toUInt16(number)</value>
<value>toUInt8(number)</value>
<value>toInt64(number)</value>
<value>toInt32(number)</value>
<value>toInt16(number)</value>
<value>toInt8(number)</value>
<value>toFloat64(number)</value>
<value>toFloat32(number)</value>
</values>
</substitution>
</substitutions>
<query>SELECT {func}({expr}) FROM numbers(1000000000) FORMAT Null</query>
</test>

View File

@ -118,6 +118,7 @@ SYSTEM FLUSH DISTRIBUTED ['FLUSH DISTRIBUTED'] TABLE SYSTEM FLUSH
SYSTEM FLUSH LOGS ['FLUSH LOGS'] GLOBAL SYSTEM FLUSH
SYSTEM FLUSH [] \N SYSTEM
SYSTEM THREAD FUZZER ['SYSTEM START THREAD FUZZER','SYSTEM STOP THREAD FUZZER','START THREAD FUZZER','STOP THREAD FUZZER'] GLOBAL SYSTEM
SYSTEM UNFREEZE ['SYSTEM UNFREEZE'] GLOBAL SYSTEM
SYSTEM [] \N ALL
dictGet ['dictHas','dictGetHierarchy','dictIsIn'] DICTIONARY ALL
addressToLine [] GLOBAL INTROSPECTION

View File

@ -22,3 +22,4 @@ command_type partition_id part_name backup_name
FREEZE PARTITION 202103 20210301_20210301_1_1_0 test_01417_single_part_old_syntax
command_type partition_id part_name backup_name
UNFREEZE PARTITION 20210301 20210301_20210301_1_1_0 test_01417_single_part_old_syntax
SYSTEM UNFREEZE 7 7_8_8_0 test_01417_single_part_7_system

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-replicated-database, no-parallel
# Tags: no-replicated-database, no-parallel, no-ordinary-database
# Tag no-replicated-database: Unsupported type of ALTER query
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
@ -57,6 +57,19 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_old_syntax UNFREEZE P
| ${CLICKHOUSE_LOCAL} --structure "$ALTER_OUT_STRUCTURE, $FREEZE_OUT_STRUCTURE" \
--query "SELECT command_type, partition_id, part_name, backup_name FROM table"
# Unfreeze the whole backup with SYSTEM query
${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze FREEZE PARTITION '7' WITH NAME 'test_01417_single_part_7_system'"
${CLICKHOUSE_CLIENT} --query "DROP TABLE table_for_freeze"
${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze UNFREEZE PARTITION '7' WITH NAME 'test_01417_single_part_7_system'" 2>/dev/null
rc=$?
if [ $rc -eq 0 ]; then
echo "ALTER query shouldn't unfreeze removed table. Code: $rc"
exit 1
fi
${CLICKHOUSE_CLIENT} --query "SYSTEM UNFREEZE WITH NAME 'test_01417_single_part_7_system'" \
| ${CLICKHOUSE_LOCAL} --structure "$ALTER_OUT_STRUCTURE, $FREEZE_OUT_STRUCTURE" \
--query "SELECT command_type, partition_id, part_name, backup_name FROM table"
# teardown
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze;"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_old_syntax;"

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