mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #63578 from ElderlyPassionFruit/add-compression-sorts-optimization
Best-effort sorting to improve compressability
This commit is contained in:
commit
46434f9040
@ -178,6 +178,10 @@ Additional parameters that control the behavior of the `MergeTree` (optional):
|
||||
|
||||
`max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting.
|
||||
|
||||
#### allow_experimental_optimized_row_order
|
||||
|
||||
`allow_experimental_optimized_row_order` - Experimental. Enables the optimization of the row order during inserts to improve the compressability of the data for compression codecs (e.g. LZ4). Analyzes and reorders the data, and thus increases the CPU overhead of inserts.
|
||||
|
||||
**Example of Sections Setting**
|
||||
|
||||
``` sql
|
||||
|
@ -885,3 +885,47 @@ Default value: false
|
||||
**See Also**
|
||||
|
||||
- [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge) setting
|
||||
|
||||
### allow_experimental_optimized_row_order
|
||||
|
||||
Controls if the row order should be optimized during inserts to improve the compressability of the newly inserted table part.
|
||||
|
||||
MergeTree tables are (optionally) compressed using [compression codecs](../../sql-reference/statements/create/table.md#column_compression_codec).
|
||||
Generic compression codecs such as LZ4 and ZSTD achieve maximum compression rates if the data exposes patterns.
|
||||
Long runs of the same value typically compress very well.
|
||||
|
||||
If this setting is enabled, ClickHouse attempts to store the data in newly inserted parts in a row order that minimizes the number of equal-value runs across the columns of the new table part.
|
||||
In other words, a small number of equal-value runs mean that individual runs are long and compress well.
|
||||
|
||||
Finding the optimal row order is computationally infeasible (NP hard).
|
||||
Therefore, ClickHouse uses a heuristics to quickly find a row order which still improves compression rates over the original row order.
|
||||
|
||||
<details markdown="1">
|
||||
|
||||
<summary>Heuristics for finding a row order</summary>
|
||||
|
||||
It is generally possible to shuffle the rows of a table (or table part) freely as SQL considers the same table (table part) in different row order equivalent.
|
||||
|
||||
This freedom of shuffling rows is restricted when a primary key is defined for the table.
|
||||
In ClickHouse, a primary key `C1, C2, ..., CN` enforces that the table rows are sorted by columns `C1`, `C2`, ... `Cn` ([clustered index](https://en.wikipedia.org/wiki/Database_index#Clustered)).
|
||||
As a result, rows can only be shuffled within "equivalence classes" of row, i.e. rows which have the same values in their primary key columns.
|
||||
The intuition is that primary keys with high-cardinality, e.g. primary keys involving a `DateTime64` timestamp column, lead to many small equivalence classes.
|
||||
Likewise, tables with a low-cardinality primary key, create few and large equivalence classes.
|
||||
A table with no primary key represents the extreme case of a single equivalence class which spans all rows.
|
||||
|
||||
The fewer and the larger the equivalence classes are, the higher the degree of freedom when re-shuffling rows.
|
||||
|
||||
The heuristics applied to find the best row order within each equivalence class is suggested by D. Lemir, O. Kaser in [Reordering columns for smaller indexes](https://doi.org/10.1016/j.ins.2011.02.002) and based on sorting the rows within each equivalence class by ascending cardinality of the non-primary key columns.
|
||||
It performs three steps:
|
||||
1. Find all equivalence classes based on the row values in primary key columns.
|
||||
2. For each equivalence class, calculate (usually estimate) the cardinalities of the non-primary-key columns.
|
||||
3. For each equivalence class, sort the rows in order of ascending non-primary-key column cardinality.
|
||||
|
||||
</details>
|
||||
|
||||
If enabled, insert operations incur additional CPU costs to analyze and optimize the row order of the new data.
|
||||
INSERTs are expected to take 30-50% longer depending on the data characteristics.
|
||||
Compression rates of LZ4 or ZSTD improve on average by 20-40%.
|
||||
|
||||
This setting works best for tables with no primary key or a low-cardinality primary key, i.e. a table with only few distinct primary key values.
|
||||
High-cardinality primary keys, e.g. involving timestamp columns of type `DateTime64`, are not expected to benefit from this setting.
|
||||
|
@ -337,7 +337,7 @@ Then, when executing the query `SELECT name FROM users_a WHERE length(name) < 5;
|
||||
|
||||
Defines storage time for values. Can be specified only for MergeTree-family tables. For the detailed description, see [TTL for columns and tables](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl).
|
||||
|
||||
## Column Compression Codecs
|
||||
## Column Compression Codecs {#column_compression_codec}
|
||||
|
||||
By default, ClickHouse applies `lz4` compression in the self-managed version, and `zstd` in ClickHouse Cloud.
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/RadixSort.h>
|
||||
#include <Common/SipHash.h>
|
||||
@ -264,6 +265,23 @@ void ColumnDecimal<T>::updatePermutation(IColumn::PermutationSortDirection direc
|
||||
}
|
||||
}
|
||||
|
||||
template <is_decimal T>
|
||||
size_t ColumnDecimal<T>::estimateCardinalityInPermutedRange(const IColumn::Permutation & permutation, const EqualRange & equal_range) const
|
||||
{
|
||||
const size_t range_size = equal_range.size();
|
||||
if (range_size <= 1)
|
||||
return range_size;
|
||||
|
||||
/// TODO use sampling if the range is too large (e.g. 16k elements, but configurable)
|
||||
HashSet<T> elements;
|
||||
for (size_t i = equal_range.from; i < equal_range.to; ++i)
|
||||
{
|
||||
size_t permuted_i = permutation[i];
|
||||
elements.insert(data[permuted_i]);
|
||||
}
|
||||
return elements.size();
|
||||
}
|
||||
|
||||
template <is_decimal T>
|
||||
ColumnPtr ColumnDecimal<T>::permute(const IColumn::Permutation & perm, size_t limit) const
|
||||
{
|
||||
|
@ -97,6 +97,8 @@ public:
|
||||
size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override;
|
||||
void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int, IColumn::Permutation & res, EqualRanges& equal_ranges) const override;
|
||||
size_t estimateCardinalityInPermutedRange(const IColumn::Permutation & permutation, const EqualRange & equal_range) const override;
|
||||
|
||||
|
||||
MutableColumnPtr cloneResized(size_t size) const override;
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/HashTable/StringHashSet.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
@ -200,6 +201,24 @@ void ColumnFixedString::updatePermutation(IColumn::PermutationSortDirection dire
|
||||
updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingStable(*this), comparator_equal, DefaultSort(), DefaultPartialSort());
|
||||
}
|
||||
|
||||
size_t ColumnFixedString::estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const
|
||||
{
|
||||
const size_t range_size = equal_range.size();
|
||||
if (range_size <= 1)
|
||||
return range_size;
|
||||
|
||||
/// TODO use sampling if the range is too large (e.g. 16k elements, but configurable)
|
||||
StringHashSet elements;
|
||||
bool inserted = false;
|
||||
for (size_t i = equal_range.from; i < equal_range.to; ++i)
|
||||
{
|
||||
size_t permuted_i = permutation[i];
|
||||
StringRef value = getDataAt(permuted_i);
|
||||
elements.emplace(value, inserted);
|
||||
}
|
||||
return elements.size();
|
||||
}
|
||||
|
||||
void ColumnFixedString::insertRangeFrom(const IColumn & src, size_t start, size_t length)
|
||||
{
|
||||
const ColumnFixedString & src_concrete = assert_cast<const ColumnFixedString &>(src);
|
||||
|
@ -142,6 +142,8 @@ public:
|
||||
void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_ranges) const override;
|
||||
|
||||
size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const override;
|
||||
|
||||
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
||||
|
||||
ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override;
|
||||
|
@ -3,9 +3,12 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include "Storages/IndicesDescription.h"
|
||||
#include "base/types.h"
|
||||
#include <base/sort.h>
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
@ -486,6 +489,21 @@ void ColumnLowCardinality::updatePermutationWithCollation(const Collator & colla
|
||||
updatePermutationImpl(limit, res, equal_ranges, comparator, equal_comparator, DefaultSort(), DefaultPartialSort());
|
||||
}
|
||||
|
||||
size_t ColumnLowCardinality::estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const
|
||||
{
|
||||
const size_t range_size = equal_range.size();
|
||||
if (range_size <= 1)
|
||||
return range_size;
|
||||
|
||||
HashSet<UInt64> elements;
|
||||
for (size_t i = equal_range.from; i < equal_range.to; ++i)
|
||||
{
|
||||
UInt64 index = getIndexes().getUInt(permutation[i]);
|
||||
elements.insert(index);
|
||||
}
|
||||
return elements.size();
|
||||
}
|
||||
|
||||
std::vector<MutableColumnPtr> ColumnLowCardinality::scatter(ColumnIndex num_columns, const Selector & selector) const
|
||||
{
|
||||
auto columns = getIndexes().scatter(num_columns, selector);
|
||||
|
@ -145,6 +145,8 @@ public:
|
||||
void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override;
|
||||
|
||||
size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const override;
|
||||
|
||||
ColumnPtr replicate(const Offsets & offsets) const override
|
||||
{
|
||||
return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().replicate(offsets));
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/HashTable/StringHashSet.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/WeakHash.h>
|
||||
@ -621,7 +622,7 @@ void ColumnNullable::updatePermutationImpl(IColumn::PermutationSortDirection dir
|
||||
if (unlikely(stability == PermutationSortStability::Stable))
|
||||
{
|
||||
for (auto & null_range : null_ranges)
|
||||
::sort(res.begin() + null_range.first, res.begin() + null_range.second);
|
||||
::sort(std::ranges::next(res.begin(), null_range.from), std::ranges::next(res.begin(), null_range.to));
|
||||
}
|
||||
|
||||
if (is_nulls_last || null_ranges.empty())
|
||||
@ -660,6 +661,33 @@ void ColumnNullable::updatePermutationWithCollation(const Collator & collator, I
|
||||
updatePermutationImpl(direction, stability, limit, null_direction_hint, res, equal_ranges, &collator);
|
||||
}
|
||||
|
||||
|
||||
size_t ColumnNullable::estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const
|
||||
{
|
||||
const size_t range_size = equal_range.size();
|
||||
if (range_size <= 1)
|
||||
return range_size;
|
||||
|
||||
/// TODO use sampling if the range is too large (e.g. 16k elements, but configurable)
|
||||
StringHashSet elements;
|
||||
bool has_null = false;
|
||||
bool inserted = false;
|
||||
for (size_t i = equal_range.from; i < equal_range.to; ++i)
|
||||
{
|
||||
size_t permuted_i = permutation[i];
|
||||
if (isNullAt(permuted_i))
|
||||
{
|
||||
has_null = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
StringRef value = getDataAt(permuted_i);
|
||||
elements.emplace(value, inserted);
|
||||
}
|
||||
}
|
||||
return elements.size() + (has_null ? 1 : 0);
|
||||
}
|
||||
|
||||
void ColumnNullable::reserve(size_t n)
|
||||
{
|
||||
getNestedColumn().reserve(n);
|
||||
|
@ -109,6 +109,7 @@ public:
|
||||
size_t limit, int null_direction_hint, Permutation & res) const override;
|
||||
void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override;
|
||||
size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const override;
|
||||
void reserve(size_t n) override;
|
||||
void shrinkToFit() override;
|
||||
void ensureOwnership() override;
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Columns/ColumnCompressed.h>
|
||||
#include <Columns/MaskOperations.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/HashTable/StringHashSet.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
@ -481,6 +482,23 @@ void ColumnString::updatePermutationWithCollation(const Collator & collator, Per
|
||||
DefaultPartialSort());
|
||||
}
|
||||
|
||||
size_t ColumnString::estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const
|
||||
{
|
||||
const size_t range_size = equal_range.size();
|
||||
if (range_size <= 1)
|
||||
return range_size;
|
||||
|
||||
/// TODO use sampling if the range is too large (e.g. 16k elements, but configurable)
|
||||
StringHashSet elements;
|
||||
bool inserted = false;
|
||||
for (size_t i = equal_range.from; i < equal_range.to; ++i)
|
||||
{
|
||||
size_t permuted_i = permutation[i];
|
||||
StringRef value = getDataAt(permuted_i);
|
||||
elements.emplace(value, inserted);
|
||||
}
|
||||
return elements.size();
|
||||
}
|
||||
|
||||
ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const
|
||||
{
|
||||
|
@ -260,6 +260,8 @@ public:
|
||||
void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const override;
|
||||
|
||||
size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const override;
|
||||
|
||||
ColumnPtr replicate(const Offsets & replicate_offsets) const override;
|
||||
|
||||
ColumnPtr compress() const override;
|
||||
|
@ -429,7 +429,7 @@ void ColumnTuple::updatePermutationImpl(IColumn::PermutationSortDirection direct
|
||||
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
while (!equal_ranges.empty() && limit && limit <= equal_ranges.back().first)
|
||||
while (!equal_ranges.empty() && limit && limit <= equal_ranges.back().from)
|
||||
equal_ranges.pop_back();
|
||||
|
||||
if (collator && column->isCollationSupported())
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/HashTable/StringHashSet.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <Common/RadixSort.h>
|
||||
#include <Common/SipHash.h>
|
||||
@ -413,6 +414,25 @@ void ColumnVector<T>::updatePermutation(IColumn::PermutationSortDirection direct
|
||||
}
|
||||
}
|
||||
|
||||
template<typename T>
|
||||
size_t ColumnVector<T>::estimateCardinalityInPermutedRange(const IColumn::Permutation & permutation, const EqualRange & equal_range) const
|
||||
{
|
||||
const size_t range_size = equal_range.size();
|
||||
if (range_size <= 1)
|
||||
return range_size;
|
||||
|
||||
/// TODO use sampling if the range is too large (e.g. 16k elements, but configurable)
|
||||
StringHashSet elements;
|
||||
bool inserted = false;
|
||||
for (size_t i = equal_range.from; i < equal_range.to; ++i)
|
||||
{
|
||||
size_t permuted_i = permutation[i];
|
||||
StringRef value = getDataAt(permuted_i);
|
||||
elements.emplace(value, inserted);
|
||||
}
|
||||
return elements.size();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
MutableColumnPtr ColumnVector<T>::cloneResized(size_t size) const
|
||||
{
|
||||
|
@ -161,6 +161,8 @@ public:
|
||||
void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_ranges) const override;
|
||||
|
||||
size_t estimateCardinalityInPermutedRange(const IColumn::Permutation & permutation, const EqualRange & equal_range) const override;
|
||||
|
||||
void reserve(size_t n) override
|
||||
{
|
||||
data.reserve_exact(n);
|
||||
|
@ -83,6 +83,11 @@ ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, const ColumnConst
|
||||
return res;
|
||||
}
|
||||
|
||||
size_t IColumn::estimateCardinalityInPermutedRange(const IColumn::Permutation & /*permutation*/, const EqualRange & equal_range) const
|
||||
{
|
||||
return equal_range.size();
|
||||
}
|
||||
|
||||
void IColumn::forEachSubcolumn(ColumnCallback callback) const
|
||||
{
|
||||
const_cast<IColumn*>(this)->forEachSubcolumn([&callback](WrappedPtr & subcolumn)
|
||||
|
@ -36,11 +36,19 @@ class Field;
|
||||
class WeakHash32;
|
||||
class ColumnConst;
|
||||
|
||||
/*
|
||||
* Represents a set of equal ranges in previous column to perform sorting in current column.
|
||||
* Used in sorting by tuples.
|
||||
* */
|
||||
using EqualRanges = std::vector<std::pair<size_t, size_t> >;
|
||||
/// A range of column values between row indexes `from` and `to`. The name "equal range" is due to table sorting as its main use case: With
|
||||
/// a PRIMARY KEY (c_pk1, c_pk2, ...), the first PK column is fully sorted. The second PK column is sorted within equal-value runs of the
|
||||
/// first PK column, and so on. The number of runs (ranges) per column increases from one primary key column to the next. An "equal range"
|
||||
/// is a run in a previous column, within the values of the current column can be sorted.
|
||||
struct EqualRange
|
||||
{
|
||||
size_t from; /// inclusive
|
||||
size_t to; /// exclusive
|
||||
EqualRange(size_t from_, size_t to_) : from(from_), to(to_) { chassert(from <= to); }
|
||||
size_t size() const { return to - from; }
|
||||
};
|
||||
|
||||
using EqualRanges = std::vector<EqualRange>;
|
||||
|
||||
/// Declares interface to store columns in memory.
|
||||
class IColumn : public COW<IColumn>
|
||||
@ -399,6 +407,9 @@ public:
|
||||
"or for Array or Tuple, containing them.");
|
||||
}
|
||||
|
||||
/// Estimate the cardinality (number of unique values) of the values in 'equal_range' after permutation, formally: |{ column[permutation[r]] : r in equal_range }|.
|
||||
virtual size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const;
|
||||
|
||||
/** Copies each element according offsets parameter.
|
||||
* (i-th element should be copied offsets[i] - offsets[i - 1] times.)
|
||||
* It is necessary in ARRAY JOIN operation.
|
||||
|
@ -139,7 +139,7 @@ void IColumn::updatePermutationImpl(
|
||||
if (equal_ranges.empty())
|
||||
return;
|
||||
|
||||
if (limit >= size() || limit > equal_ranges.back().second)
|
||||
if (limit >= size() || limit > equal_ranges.back().to)
|
||||
limit = 0;
|
||||
|
||||
EqualRanges new_ranges;
|
||||
|
@ -166,7 +166,7 @@ void getBlockSortPermutationImpl(const Block & block, const SortDescription & de
|
||||
|
||||
for (const auto & column_with_sort_description : columns_with_sort_descriptions)
|
||||
{
|
||||
while (!ranges.empty() && limit && limit <= ranges.back().first)
|
||||
while (!ranges.empty() && limit && limit <= ranges.back().from)
|
||||
ranges.pop_back();
|
||||
|
||||
if (ranges.empty())
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataWriter.h>
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/RowOrderOptimizer.h>
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
@ -502,6 +503,12 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocksAlreadySorted);
|
||||
}
|
||||
|
||||
if (data.getSettings()->allow_experimental_optimized_row_order)
|
||||
{
|
||||
RowOrderOptimizer::optimize(block, sort_description, perm);
|
||||
perm_ptr = &perm;
|
||||
}
|
||||
|
||||
Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names;
|
||||
if (context->getSettingsRef().optimize_on_insert)
|
||||
{
|
||||
@ -722,6 +729,12 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl(
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterBlocksAlreadySorted);
|
||||
}
|
||||
|
||||
if (data.getSettings()->allow_experimental_optimized_row_order)
|
||||
{
|
||||
RowOrderOptimizer::optimize(block, sort_description, perm);
|
||||
perm_ptr = &perm;
|
||||
}
|
||||
|
||||
if (projection.type == ProjectionDescription::Type::Aggregate && merge_is_needed)
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::MergeTreeDataProjectionWriterMergingBlocksMicroseconds);
|
||||
|
@ -199,6 +199,7 @@ struct Settings;
|
||||
M(Bool, cache_populated_by_fetch, false, "Only available in ClickHouse Cloud", 0) \
|
||||
M(Bool, force_read_through_cache_for_merges, false, "Force read-through filesystem cache for merges", 0) \
|
||||
M(Bool, allow_experimental_replacing_merge_with_cleanup, false, "Allow experimental CLEANUP merges for ReplacingMergeTree with is_deleted column.", 0) \
|
||||
M(Bool, allow_experimental_optimized_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \
|
||||
\
|
||||
/** Compress marks and primary key. */ \
|
||||
M(Bool, compress_marks, true, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \
|
||||
|
184
src/Storages/MergeTree/RowOrderOptimizer.cpp
Normal file
184
src/Storages/MergeTree/RowOrderOptimizer.cpp
Normal file
@ -0,0 +1,184 @@
|
||||
#include <Storages/MergeTree/RowOrderOptimizer.h>
|
||||
|
||||
#include <Interpreters/sortBlock.h>
|
||||
#include <base/sort.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <numeric>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Do the left and right row contain equal values in the sorting key columns (usually the primary key columns)
|
||||
bool haveEqualSortingKeyValues(const Block & block, const SortDescription & sort_description, size_t left_row, size_t right_row)
|
||||
{
|
||||
for (const auto & sort_column : sort_description)
|
||||
{
|
||||
const String & sort_col = sort_column.column_name;
|
||||
const IColumn & column = *block.getByName(sort_col).column;
|
||||
if (column.compareAt(left_row, right_row, column, 1) != 0)
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/// Returns the sorted indexes of all non-sorting-key columns.
|
||||
std::vector<size_t> getOtherColumnIndexes(const Block & block, const SortDescription & sort_description)
|
||||
{
|
||||
const size_t sorting_key_columns_count = sort_description.size();
|
||||
const size_t all_columns_count = block.columns();
|
||||
|
||||
std::vector<size_t> other_column_indexes;
|
||||
other_column_indexes.reserve(all_columns_count - sorting_key_columns_count);
|
||||
|
||||
if (sorting_key_columns_count == 0)
|
||||
{
|
||||
other_column_indexes.resize(block.columns());
|
||||
iota(other_column_indexes.begin(), other_column_indexes.end(), 0);
|
||||
}
|
||||
else
|
||||
{
|
||||
std::vector<size_t> sorted_column_indexes;
|
||||
sorted_column_indexes.reserve(sorting_key_columns_count);
|
||||
for (const SortColumnDescription & sort_column : sort_description)
|
||||
{
|
||||
size_t idx = block.getPositionByName(sort_column.column_name);
|
||||
sorted_column_indexes.emplace_back(idx);
|
||||
}
|
||||
::sort(sorted_column_indexes.begin(), sorted_column_indexes.end());
|
||||
|
||||
std::vector<size_t> all_column_indexes(all_columns_count);
|
||||
std::iota(all_column_indexes.begin(), all_column_indexes.end(), 0);
|
||||
std::set_difference(
|
||||
all_column_indexes.begin(),
|
||||
all_column_indexes.end(),
|
||||
sorted_column_indexes.begin(),
|
||||
sorted_column_indexes.end(),
|
||||
std::back_inserter(other_column_indexes));
|
||||
}
|
||||
chassert(other_column_indexes.size() == all_columns_count - sorting_key_columns_count);
|
||||
return other_column_indexes;
|
||||
}
|
||||
|
||||
/// Returns a set of equal row ranges (equivalence classes) with the same row values for all sorting key columns (usually primary key columns.)
|
||||
/// Example with 2 PK columns, 2 other columns --> 3 equal ranges
|
||||
/// pk1 pk2 c1 c2
|
||||
/// ----------------------
|
||||
/// 1 1 a b
|
||||
/// 1 1 b e
|
||||
/// --------
|
||||
/// 1 2 e a
|
||||
/// 1 2 d c
|
||||
/// 1 2 e a
|
||||
/// --------
|
||||
/// 2 1 a 3
|
||||
/// ----------------------
|
||||
EqualRanges getEqualRanges(const Block & block, const SortDescription & sort_description, const IColumn::Permutation & permutation, const LoggerPtr & log)
|
||||
{
|
||||
LOG_TRACE(log, "Finding equal ranges");
|
||||
EqualRanges ranges;
|
||||
const size_t rows = block.rows();
|
||||
if (sort_description.empty())
|
||||
{
|
||||
ranges.push_back({0, rows});
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < rows;)
|
||||
{
|
||||
size_t j = i;
|
||||
while (j < rows && haveEqualSortingKeyValues(block, sort_description, permutation[i], permutation[j]))
|
||||
++j;
|
||||
ranges.push_back({i, j});
|
||||
i = j;
|
||||
}
|
||||
}
|
||||
return ranges;
|
||||
}
|
||||
|
||||
std::vector<size_t> getCardinalitiesInPermutedRange(
|
||||
const Block & block,
|
||||
const std::vector<size_t> & other_column_indexes,
|
||||
const IColumn::Permutation & permutation,
|
||||
const EqualRange & equal_range)
|
||||
{
|
||||
std::vector<size_t> cardinalities(other_column_indexes.size());
|
||||
for (size_t i = 0; i < other_column_indexes.size(); ++i)
|
||||
{
|
||||
const size_t column_id = other_column_indexes[i];
|
||||
const ColumnPtr & column = block.getByPosition(column_id).column;
|
||||
cardinalities[i] = column->estimateCardinalityInPermutedRange(permutation, equal_range);
|
||||
}
|
||||
return cardinalities;
|
||||
}
|
||||
|
||||
void updatePermutationInEqualRange(
|
||||
const Block & block,
|
||||
const std::vector<size_t> & other_column_indexes,
|
||||
IColumn::Permutation & permutation,
|
||||
const EqualRange & equal_range,
|
||||
const std::vector<size_t> & cardinalities)
|
||||
{
|
||||
LoggerPtr log = getLogger("RowOrderOptimizer");
|
||||
|
||||
LOG_TRACE(log, "Starting optimization in equal range");
|
||||
|
||||
std::vector<size_t> column_order(other_column_indexes.size());
|
||||
iota(column_order.begin(), column_order.end(), 0);
|
||||
auto cmp = [&](size_t lhs, size_t rhs) -> bool { return cardinalities[lhs] < cardinalities[rhs]; };
|
||||
stable_sort(column_order.begin(), column_order.end(), cmp);
|
||||
|
||||
std::vector<EqualRange> ranges = {equal_range};
|
||||
LOG_TRACE(log, "equal_range: .from: {}, .to: {}", equal_range.from, equal_range.to);
|
||||
for (size_t i : column_order)
|
||||
{
|
||||
const size_t column_id = other_column_indexes[i];
|
||||
const ColumnPtr & column = block.getByPosition(column_id).column;
|
||||
LOG_TRACE(log, "i: {}, column_id: {}, column->getName(): {}, cardinality: {}", i, column_id, column->getName(), cardinalities[i]);
|
||||
column->updatePermutation(
|
||||
IColumn::PermutationSortDirection::Ascending, IColumn::PermutationSortStability::Stable, 0, 1, permutation, ranges);
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Finish optimization in equal range");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void RowOrderOptimizer::optimize(const Block & block, const SortDescription & sort_description, IColumn::Permutation & permutation)
|
||||
{
|
||||
LoggerPtr log = getLogger("RowOrderOptimizer");
|
||||
|
||||
LOG_TRACE(log, "Starting optimization");
|
||||
|
||||
if (block.columns() == 0)
|
||||
return; /// a table without columns, this should not happen in the first place ...
|
||||
|
||||
if (permutation.empty())
|
||||
{
|
||||
const size_t rows = block.rows();
|
||||
permutation.resize(rows);
|
||||
iota(permutation.data(), rows, IColumn::Permutation::value_type(0));
|
||||
}
|
||||
|
||||
const EqualRanges equal_ranges = getEqualRanges(block, sort_description, permutation, log);
|
||||
const std::vector<size_t> other_columns_indexes = getOtherColumnIndexes(block, sort_description);
|
||||
|
||||
LOG_TRACE(log, "block.columns(): {}, block.rows(): {}, sort_description.size(): {}, equal_ranges.size(): {}", block.columns(), block.rows(), sort_description.size(), equal_ranges.size());
|
||||
|
||||
for (const auto & equal_range : equal_ranges)
|
||||
{
|
||||
if (equal_range.size() <= 1)
|
||||
continue;
|
||||
const std::vector<size_t> cardinalities = getCardinalitiesInPermutedRange(block, other_columns_indexes, permutation, equal_range);
|
||||
updatePermutationInEqualRange(block, other_columns_indexes, permutation, equal_range, cardinalities);
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Finished optimization");
|
||||
}
|
||||
|
||||
}
|
26
src/Storages/MergeTree/RowOrderOptimizer.h
Normal file
26
src/Storages/MergeTree/RowOrderOptimizer.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/SortDescription.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class RowOrderOptimizer
|
||||
{
|
||||
public:
|
||||
/// Given the columns in a Block with a sub-set of them as sorting key columns (usually primary key columns --> SortDescription), and a
|
||||
/// permutation of the rows, this function tries to "improve" the permutation such that the data can be compressed better by generic
|
||||
/// compression algorithms such as zstd. The heuristics is based on D. Lemire, O. Kaser (2011): Reordering columns for smaller
|
||||
/// indexes, https://doi.org/10.1016/j.ins.2011.02.002
|
||||
/// The algorithm works like this:
|
||||
/// - Divide the sorting key columns horizontally into "equal ranges". An equal range is defined by the same sorting key values on all
|
||||
/// of its rows. We can re-shuffle the non-sorting-key values within each equal range freely.
|
||||
/// - Determine (estimate) for each equal range the cardinality of each non-sorting-key column.
|
||||
/// - The simple heuristics applied is that non-sorting key columns will be sorted (within each equal range) in order of ascending
|
||||
/// cardinality. This maximizes the length of equal-value runs within the non-sorting-key columns, leading to better compressability.
|
||||
static void optimize(const Block & block, const SortDescription & sort_description, IColumn::Permutation & permutation);
|
||||
};
|
||||
|
||||
}
|
@ -0,0 +1,78 @@
|
||||
Simple test
|
||||
Egor 1
|
||||
Egor 2
|
||||
Igor 1
|
||||
Igor 2
|
||||
Igor 3
|
||||
Cardinalities test
|
||||
Alex 1 63 0
|
||||
Alex 1 65 0
|
||||
Alex 1 239 0
|
||||
Alex 2 224 0
|
||||
Alex 4 83 0
|
||||
Alex 4 134 0
|
||||
Alex 4 192 0
|
||||
Bob 2 53 0
|
||||
Bob 4 100 0
|
||||
Bob 4 177 0
|
||||
Bob 4 177 0
|
||||
Nikita 1 173 0
|
||||
Nikita 1 228 0
|
||||
Nikita 2 148 0
|
||||
Nikita 2 148 0
|
||||
Nikita 2 208 0
|
||||
Alex 1 63 1
|
||||
Alex 1 65 1
|
||||
Alex 1 239 1
|
||||
Alex 2 128 1
|
||||
Alex 2 128 1
|
||||
Alex 2 224 1
|
||||
Alex 4 83 1
|
||||
Alex 4 83 1
|
||||
Alex 4 134 1
|
||||
Alex 4 134 1
|
||||
Alex 4 192 1
|
||||
Bob 2 53 1
|
||||
Bob 2 53 1
|
||||
Bob 2 187 1
|
||||
Bob 2 187 1
|
||||
Bob 4 100 1
|
||||
Nikita 1 173 1
|
||||
Nikita 1 228 1
|
||||
Nikita 2 54 1
|
||||
Nikita 2 54 1
|
||||
Nikita 2 148 1
|
||||
Nikita 2 208 1
|
||||
Equivalence classes test
|
||||
AB 1 9.81 0
|
||||
A\0 0 2.7 1
|
||||
A\0 1 2.7 1
|
||||
B\0 0 2.7 1
|
||||
B\0 1 2.7 1
|
||||
A\0 1 42 1
|
||||
B\0 0 42 1
|
||||
A\0 0 3.14 \N
|
||||
B\0 -1 3.14 \N
|
||||
B\0 2 3.14 \N
|
||||
AB 0 42 \N
|
||||
AB 0 42 \N
|
||||
B\0 0 42 \N
|
||||
A\0 1 42 \N
|
||||
A\0 1 42 \N
|
||||
B\0 1 42 \N
|
||||
Many types test
|
||||
A\0\0\0\0\0 2020-01-01 [0,1.1] 10 some string {'key':'value'} (123)
|
||||
A\0\0\0\0\0 2020-01-01 [0,1.1] \N example {} (26)
|
||||
A\0\0\0\0\0 2020-01-01 [2.2,1.1] 1 some other string {'key2':'value2'} (5)
|
||||
A\0\0\0\0\0 2020-01-02 [2.2,1.1] 1 some other string {'key2':'value2'} (5)
|
||||
A\0\0\0\0\0 2020-01-02 [0,1.1] 10 some string {'key':'value'} (123)
|
||||
A\0\0\0\0\0 2020-01-02 [0,2.2] 10 example {} (26)
|
||||
B\0\0\0\0\0 2020-01-04 [0,2.2] \N example {} (26)
|
||||
B\0\0\0\0\0 2020-01-04 [0,1.1] 10 some string {'key':'value'} (123)
|
||||
B\0\0\0\0\0 2020-01-04 [2.2,1.1] 1 some string {'key2':'value2'} (5)
|
||||
B\0\0\0\0\0 2020-01-05 [0,1.1] 10 some string {'key':'value'} (123)
|
||||
B\0\0\0\0\0 2020-01-05 [0,2.2] \N example {} (26)
|
||||
B\0\0\0\0\0 2020-01-05 [2.2,1.1] 1 some other string {'key':'value'} (5)
|
||||
C\0\0\0\0\0 2020-01-04 [0,1.1] 10 some string {'key':'value'} (5)
|
||||
C\0\0\0\0\0 2020-01-04 [0,2.2] \N example {} (26)
|
||||
C\0\0\0\0\0 2020-01-04 [2.2,1.1] 1 some other string {'key2':'value2'} (5)
|
@ -0,0 +1,98 @@
|
||||
-- Checks that no bad things happen when the table optimizes the row order to improve compressability during insert.
|
||||
|
||||
|
||||
-- Below SELECTs intentionally only ORDER BY the table primary key and rely on read-in-order optimization
|
||||
SET optimize_read_in_order = 1;
|
||||
|
||||
-- Just simple check, that optimization works correctly for table with 2 columns and 2 equivalence classes.
|
||||
SELECT 'Simple test';
|
||||
|
||||
DROP TABLE IF EXISTS tab;
|
||||
|
||||
CREATE TABLE tab (
|
||||
name String,
|
||||
event Int8
|
||||
) ENGINE = MergeTree
|
||||
ORDER BY name
|
||||
SETTINGS allow_experimental_optimized_row_order = true;
|
||||
INSERT INTO tab VALUES ('Igor', 3), ('Egor', 1), ('Egor', 2), ('Igor', 2), ('Igor', 1);
|
||||
|
||||
SELECT * FROM tab ORDER BY name SETTINGS max_threads=1;
|
||||
|
||||
DROP TABLE tab;
|
||||
|
||||
-- Checks that RowOptimizer correctly selects the order for columns according to cardinality, with an empty ORDER BY.
|
||||
-- There are 4 columns with cardinalities {name : 3, timestamp": 3, money: 17, flag: 2}, so the columns order must be {flag, name, timestamp, money}.
|
||||
SELECT 'Cardinalities test';
|
||||
|
||||
DROP TABLE IF EXISTS tab;
|
||||
|
||||
CREATE TABLE tab (
|
||||
name String,
|
||||
timestamp Int64,
|
||||
money UInt8,
|
||||
flag String
|
||||
) ENGINE = MergeTree
|
||||
ORDER BY ()
|
||||
SETTINGS allow_experimental_optimized_row_order = True;
|
||||
INSERT INTO tab VALUES ('Bob', 4, 100, '1'), ('Nikita', 2, 54, '1'), ('Nikita', 1, 228, '1'), ('Alex', 4, 83, '1'), ('Alex', 4, 134, '1'), ('Alex', 1, 65, '0'), ('Alex', 4, 134, '1'), ('Bob', 2, 53, '0'), ('Alex', 4, 83, '0'), ('Alex', 1, 63, '1'), ('Bob', 2, 53, '1'), ('Alex', 4, 192, '1'), ('Alex', 2, 128, '1'), ('Nikita', 2, 148, '0'), ('Bob', 4, 177, '0'), ('Nikita', 1, 173, '0'), ('Alex', 1, 239, '0'), ('Alex', 1, 63, '0'), ('Alex', 2, 224, '1'), ('Bob', 4, 177, '0'), ('Alex', 2, 128, '1'), ('Alex', 4, 134, '0'), ('Alex', 4, 83, '1'), ('Bob', 4, 100, '0'), ('Nikita', 2, 54, '1'), ('Alex', 1, 239, '1'), ('Bob', 2, 187, '1'), ('Alex', 1, 65, '1'), ('Bob', 2, 53, '1'), ('Alex', 2, 224, '0'), ('Alex', 4, 192, '0'), ('Nikita', 1, 173, '1'), ('Nikita', 2, 148, '1'), ('Bob', 2, 187, '1'), ('Nikita', 2, 208, '1'), ('Nikita', 2, 208, '0'), ('Nikita', 1, 228, '0'), ('Nikita', 2, 148, '0');
|
||||
|
||||
SELECT * FROM tab SETTINGS max_threads=1;
|
||||
|
||||
DROP TABLE tab;
|
||||
|
||||
-- Checks that RowOptimizer correctly selects the order for columns according to cardinality in each equivalence class obtained using SortDescription.
|
||||
-- There are two columns in the SortDescription: {flag, money} in this order.
|
||||
-- So there are 5 equivalence classes: {9.81, 9}, {2.7, 1}, {42, 1}, {3.14, Null}, {42, Null}.
|
||||
-- For the first three of them cardinalities of the other 2 columns are equal, so they are sorted in order {0, 1} in these classes.
|
||||
-- In the fourth class cardinalities: {name : 2, timestamp : 3}, so they are sorted in order {name, timestamp} in this class.
|
||||
-- In the fifth class cardinalities: {name : 3, timestamp : 2}, so they are sorted in order {timestamp, name} in this class.
|
||||
SELECT 'Equivalence classes test';
|
||||
|
||||
DROP TABLE IF EXISTS tab;
|
||||
|
||||
CREATE TABLE tab (
|
||||
name FixedString(2),
|
||||
timestamp Float32,
|
||||
money Float64,
|
||||
flag Nullable(Int32)
|
||||
) ENGINE = MergeTree
|
||||
ORDER BY (flag, money)
|
||||
SETTINGS allow_experimental_optimized_row_order = True, allow_nullable_key = True;
|
||||
INSERT INTO tab VALUES ('AB', 0, 42, Null), ('AB', 0, 42, Null), ('A', 1, 42, Null), ('AB', 1, 9.81, 0), ('B', 0, 42, Null), ('B', -1, 3.14, Null), ('B', 1, 2.7, 1), ('B', 0, 42, 1), ('A', 1, 42, 1), ('B', 1, 42, Null), ('B', 0, 2.7, 1), ('A', 0, 2.7, 1), ('B', 2, 3.14, Null), ('A', 0, 3.14, Null), ('A', 1, 2.7, 1), ('A', 1, 42, Null);
|
||||
|
||||
SELECT * FROM tab ORDER BY (flag, money) SETTINGS max_threads=1;
|
||||
|
||||
DROP TABLE tab;
|
||||
|
||||
-- Checks that no bad things happen when the table optimizes the row order to improve compressability during insert for many different column types.
|
||||
-- For some of these types estimateCardinalityInPermutedRange returns just the size of the current equal range.
|
||||
-- There are 5 equivalence classes, each of them has equal size = 3.
|
||||
-- In the first of them cardinality of the vector_array column equals 2, other cardinalities equals 3.
|
||||
-- In the second of them cardinality of the nullable_int column equals 2, other cardinalities equals 3.
|
||||
-- ...
|
||||
-- In the fifth of them cardinality of the tuple_column column equals 2, other cardinalities equals 3.
|
||||
-- So, for all of this classes for columns with cardinality equals 2 such that estimateCardinalityInPermutedRange methid is implemented,
|
||||
-- this column must be the first in the column order, all others must be in the stable order.
|
||||
-- For all other classes columns must be in the stable order.
|
||||
SELECT 'Many types test';
|
||||
|
||||
DROP TABLE IF EXISTS tab;
|
||||
|
||||
CREATE TABLE tab (
|
||||
fixed_str FixedString(6),
|
||||
event_date Date,
|
||||
vector_array Array(Float32),
|
||||
nullable_int Nullable(Int128),
|
||||
low_card_string LowCardinality(String),
|
||||
map_column Map(String, String),
|
||||
tuple_column Tuple(UInt256)
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY (fixed_str, event_date)
|
||||
SETTINGS allow_experimental_optimized_row_order = True;
|
||||
|
||||
INSERT INTO tab VALUES ('A', '2020-01-01', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('A', '2020-01-01', [0.0, 1.1], NULL, 'example', {}, (26)), ('A', '2020-01-01', [2.2, 1.1], 1, 'some other string', {'key2':'value2'}, (5)), ('A', '2020-01-02', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('A', '2020-01-02', [0.0, 2.2], 10, 'example', {}, (26)), ('A', '2020-01-02', [2.2, 1.1], 1, 'some other string', {'key2':'value2'}, (5)), ('B', '2020-01-04', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('B', '2020-01-04', [0.0, 2.2], Null, 'example', {}, (26)), ('B', '2020-01-04', [2.2, 1.1], 1, 'some string', {'key2':'value2'}, (5)), ('B', '2020-01-05', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('B', '2020-01-05', [0.0, 2.2], Null, 'example', {}, (26)), ('B', '2020-01-05', [2.2, 1.1], 1, 'some other string', {'key':'value'}, (5)), ('C', '2020-01-04', [0.0, 1.1], 10, 'some string', {'key':'value'}, (5)), ('C', '2020-01-04', [0.0, 2.2], Null, 'example', {}, (26)), ('C', '2020-01-04', [2.2, 1.1], 1, 'some other string', {'key2':'value2'}, (5));
|
||||
|
||||
SELECT * FROM tab ORDER BY (fixed_str, event_date) SETTINGS max_threads=1;
|
||||
|
||||
DROP TABLE tab;
|
@ -446,6 +446,7 @@ KafkaLibrdkafkaThreads
|
||||
KafkaProducers
|
||||
KafkaWrites
|
||||
Kahan
|
||||
Kaser
|
||||
KeeperAliveConnections
|
||||
KeeperMap
|
||||
KeeperOutstandingRequets
|
||||
@ -466,6 +467,7 @@ LOCALTIME
|
||||
LOCALTIMESTAMP
|
||||
LONGLONG
|
||||
LOONGARCH
|
||||
Lemir
|
||||
Levenshtein
|
||||
Liao
|
||||
LibFuzzer
|
||||
|
Loading…
Reference in New Issue
Block a user